From e595c8d08a20a122295af62d5e9cc4116f9727f6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 23 Oct 2014 17:20:00 -0700 Subject: [PATCH 001/115] [SPARK-3993] [PySpark] fix bug while reuse worker after take() After take(), maybe there are some garbage left in the socket, then next task assigned to this worker will hang because of corrupted data. We should make sure the socket is clean before reuse it, write END_OF_STREAM at the end, and check it after read out all result from python. Author: Davies Liu Author: Davies Liu Closes #2838 from davies/fix_reuse and squashes the following commits: 8872914 [Davies Liu] fix tests 660875b [Davies Liu] fix bug while reuse worker after take() --- .../scala/org/apache/spark/SparkEnv.scala | 2 ++ .../apache/spark/api/python/PythonRDD.scala | 11 ++++++++++- python/pyspark/daemon.py | 5 ++++- python/pyspark/serializers.py | 1 + python/pyspark/tests.py | 19 ++++++++++++++++++- python/pyspark/worker.py | 11 +++++++++-- 6 files changed, 44 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index aba713cb4267a..906a00b0bd17c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -68,6 +68,7 @@ class SparkEnv ( val shuffleMemoryManager: ShuffleMemoryManager, val conf: SparkConf) extends Logging { + private[spark] var isStopped = false private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation @@ -75,6 +76,7 @@ class SparkEnv ( private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() private[spark] def stop() { + isStopped = true pythonWorkers.foreach { case(key, worker) => worker.stop() } Option(httpFileServer).foreach(_.stop()) mapOutputTracker.stop() diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 29ca751519abd..163dca6cade5a 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -75,6 +75,7 @@ private[spark] class PythonRDD( var complete_cleanly = false context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() + writerThread.join() if (reuse_worker && complete_cleanly) { env.releasePythonWorker(pythonExec, envVars.toMap, worker) } else { @@ -145,7 +146,9 @@ private[spark] class PythonRDD( stream.readFully(update) accumulator += Collections.singletonList(update) } - complete_cleanly = true + if (stream.readInt() == SpecialLengths.END_OF_STREAM) { + complete_cleanly = true + } null } } catch { @@ -154,6 +157,10 @@ private[spark] class PythonRDD( logDebug("Exception thrown after task interruption", e) throw new TaskKilledException + case e: Exception if env.isStopped => + logDebug("Exception thrown after context is stopped", e) + null // exit silently + case e: Exception if writerThread.exception.isDefined => logError("Python worker exited unexpectedly (crashed)", e) logError("This may have been caused by a prior exception:", writerThread.exception.get) @@ -235,6 +242,7 @@ private[spark] class PythonRDD( // Data values PythonRDD.writeIteratorToStream(firstParent.iterator(split, context), dataOut) dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) + dataOut.writeInt(SpecialLengths.END_OF_STREAM) dataOut.flush() } catch { case e: Exception if context.isCompleted || context.isInterrupted => @@ -306,6 +314,7 @@ private object SpecialLengths { val END_OF_DATA_SECTION = -1 val PYTHON_EXCEPTION_THROWN = -2 val TIMING_DATA = -3 + val END_OF_STREAM = -4 } private[spark] object PythonRDD extends Logging { diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 64d6202acb27d..dbb34775d9ac5 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -26,7 +26,7 @@ import gc from errno import EINTR, ECHILD, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN -from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN +from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT from pyspark.worker import main as worker_main from pyspark.serializers import read_int, write_int @@ -46,6 +46,9 @@ def worker(sock): signal.signal(SIGHUP, SIG_DFL) signal.signal(SIGCHLD, SIG_DFL) signal.signal(SIGTERM, SIG_DFL) + # restore the handler for SIGINT, + # it's useful for debugging (show the stacktrace before exit) + signal.signal(SIGINT, signal.default_int_handler) # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 08a0f0d8ffb3e..904bd9f2652d3 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -80,6 +80,7 @@ class SpecialLengths(object): END_OF_DATA_SECTION = -1 PYTHON_EXCEPTION_THROWN = -2 TIMING_DATA = -3 + END_OF_STREAM = -4 class Serializer(object): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 1a8e4150e63c3..7a2107ec326ee 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -31,7 +31,7 @@ import time import zipfile import random -from platform import python_implementation +import threading if sys.version_info[:2] <= (2, 6): try: @@ -1380,6 +1380,23 @@ def test_accumulator_when_reuse_worker(self): self.assertEqual(sum(range(100)), acc2.value) self.assertEqual(sum(range(100)), acc1.value) + def test_reuse_worker_after_take(self): + rdd = self.sc.parallelize(range(100000), 1) + self.assertEqual(0, rdd.first()) + + def count(): + try: + rdd.count() + except Exception: + pass + + t = threading.Thread(target=count) + t.daemon = True + t.start() + t.join(5) + self.assertTrue(not t.isAlive()) + self.assertEqual(100000, rdd.count()) + class SparkSubmitTests(unittest.TestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8257dddfee1c3..2bdccb5e93f09 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -57,7 +57,7 @@ def main(infile, outfile): boot_time = time.time() split_index = read_int(infile) if split_index == -1: # for unit tests - return + exit(-1) # initialize global state shuffle.MemoryBytesSpilled = 0 @@ -111,7 +111,6 @@ def process(): try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) write_with_length(traceback.format_exc(), outfile) - outfile.flush() except IOError: # JVM close the socket pass @@ -131,6 +130,14 @@ def process(): for (aid, accum) in _accumulatorRegistry.items(): pickleSer._write_with_length((aid, accum._value), outfile) + # check end of stream + if read_int(infile) == SpecialLengths.END_OF_STREAM: + write_int(SpecialLengths.END_OF_STREAM, outfile) + else: + # write a different value to tell JVM to not reuse this worker + write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) + exit(-1) + if __name__ == '__main__': # Read a local port to connect to from stdin From a29c9bd614e6d8088f759ae39218b945604d0b5b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 23 Oct 2014 22:15:03 -0700 Subject: [PATCH 002/115] [SPARK-4000][BUILD] Sends archived unit tests logs to Jenkins master This PR sends archived unit tests logs to the build history directory in Jenkins master, so that we can serve it via HTTP later to help debugging Jenkins build failures. pwendell JoshRosen Please help review, thanks! Author: Cheng Lian Closes #2845 from liancheng/log-archive and squashes the following commits: ac8d9d4 [Cheng Lian] Includes build number in messages posted to GitHub 68c7010 [Cheng Lian] Logs backup should be implemented in dev/run-tests-jenkins 4b912f7 [Cheng Lian] Sends archived unit tests logs to Jenkins master --- dev/run-tests-jenkins | 53 +++++++++++++++++++++++++++++++++---------- 1 file changed, 41 insertions(+), 12 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 451f3b771cc76..87c6715153da7 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -53,9 +53,9 @@ function post_message () { local message=$1 local data="{\"body\": \"$message\"}" local HTTP_CODE_HEADER="HTTP Response Code: " - + echo "Attempting to post to Github..." - + local curl_output=$( curl `#--dump-header -` \ --silent \ @@ -75,12 +75,12 @@ function post_message () { echo " > data: ${data}" >&2 # exit $curl_status fi - + local api_response=$( echo "${curl_output}" \ | grep -v -e "^${HTTP_CODE_HEADER}" ) - + local http_code=$( echo "${curl_output}" \ | grep -e "^${HTTP_CODE_HEADER}" \ @@ -92,12 +92,39 @@ function post_message () { echo " > api_response: ${api_response}" >&2 echo " > data: ${data}" >&2 fi - + if [ "$curl_status" -eq 0 ] && [ "$http_code" -eq "201" ]; then echo " > Post successful." fi } +function send_archived_logs () { + echo "Archiving unit tests logs..." + + local log_files=$(find . -name "unit-tests.log") + + if [ -z "$log_files" ]; then + echo "> No log files found." >&2 + else + local log_archive="unit-tests-logs.tar.gz" + echo "$log_files" | xargs tar czf ${log_archive} + + local jenkins_build_dir=${JENKINS_HOME}/jobs/${JOB_NAME}/builds/${BUILD_NUMBER} + local scp_output=$(scp ${log_archive} amp-jenkins-master:${jenkins_build_dir}/${log_archive}) + local scp_status="$?" + + if [ "$scp_status" -ne 0 ]; then + echo "Failed to send archived unit tests logs to Jenkins master." >&2 + echo "> scp_status: ${scp_status}" >&2 + echo "> scp_output: ${scp_output}" >&2 + else + echo "> Send successful." + fi + + rm -f ${log_archive} + fi +} + # We diff master...$ghprbActualCommit because that gets us changes introduced in the PR #+ and not anything else added to master since the PR was branched. @@ -109,7 +136,7 @@ function post_message () { else merge_note=" * This patch merges cleanly." fi - + source_files=$( git diff master...$ghprbActualCommit --name-only `# diff patch against master from branch point` \ | grep -v -e "\/test" `# ignore files in test directories` \ @@ -144,12 +171,12 @@ function post_message () { # post start message { start_message="\ - [QA tests have started](${BUILD_URL}consoleFull) for \ + [Test build ${BUILD_DISPLAY_NAME} has started](${BUILD_URL}consoleFull) for \ PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." - + start_message="${start_message}\n${merge_note}" # start_message="${start_message}\n${public_classes_note}" - + post_message "$start_message" } @@ -159,7 +186,7 @@ function post_message () { test_result="$?" if [ "$test_result" -eq "124" ]; then - fail_message="**[Tests timed out](${BUILD_URL}consoleFull)** \ + fail_message="**[Test build ${BUILD_DISPLAY_NAME} timed out](${BUILD_URL}consoleFull)** \ for PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL}) \ after a configured wait of \`${TESTS_TIMEOUT}\`." @@ -187,15 +214,17 @@ function post_message () { else failing_test="some tests" fi - + test_result_note=" * This patch **fails $failing_test**." fi + + send_archived_logs } # post end message { result_message="\ - [QA tests have finished](${BUILD_URL}consoleFull) for \ + [Test build ${BUILD_DISPLAY_NAME} has finished](${BUILD_URL}consoleFull) for \ PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." result_message="${result_message}\n${test_result_note}" From 0aea22895ce94f4f80faad5a2c41b76edac172b9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 23 Oct 2014 23:49:50 -0700 Subject: [PATCH 003/115] SPARK-3812 Build changes to publish effective pom. Author: Prashant Sharma Closes #2921 from ScrapCodes/build-changes-effective-pom and squashes the following commits: 8841491 [Prashant Sharma] Fixed broken maven build. aa7b91d [Prashant Sharma] used an unused dep. 0300dac [Prashant Sharma] improved comment messages.. 28f891e [Prashant Sharma] Added a useless dependency, so that we can shade it. And realized fake shading works for us. 553d96b [Prashant Sharma] Shaded some unused class of an unused dep, to generate effective pom(s) --- pom.xml | 33 ++++++++++++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a7e71f9ca5596..a9897b866b036 100644 --- a/pom.xml +++ b/pom.xml @@ -248,7 +248,17 @@ - + + + + org.spark-project.spark + unused + 1.0.0 + + @@ -997,6 +1007,27 @@ + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + org.spark-project.spark:unused + + + + + + package + + shade + + + + org.apache.maven.plugins maven-enforcer-plugin From 809c785bcc33e684a68ea14240a466def864199a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 23 Oct 2014 23:58:00 -0700 Subject: [PATCH 004/115] [SPARK-2652] [PySpark] donot use KyroSerializer as default serializer KyroSerializer can not serialize customized class without registered explicitly, use it as default serializer in PySpark will introduce some regression in MLlib. cc mengxr Author: Davies Liu Closes #2916 from davies/revert and squashes the following commits: 43eb6d3 [Davies Liu] donot use KyroSerializer as default serializer --- python/pyspark/context.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 8d27ccb95f82c..5f8dcedb1eea2 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -43,7 +43,6 @@ # These are special default configs for PySpark, they will overwrite # the default ones for Spark if they are not configured by user. DEFAULT_CONFIGS = { - "spark.serializer": "org.apache.spark.serializer.KryoSerializer", "spark.serializer.objectStreamReset": 100, "spark.rdd.compress": True, } From d2987e8f7a2cb3bf971f381399d8efdccb51d3d2 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 24 Oct 2014 08:51:08 -0500 Subject: [PATCH 005/115] [SPARK-3900][YARN] ApplicationMaster's shutdown hook fails and IllegalStateException is thrown. ApplicationMaster registers a shutdown hook and it calls ApplicationMaster#cleanupStagingDir. cleanupStagingDir invokes FileSystem.get(yarnConf) and it invokes FileSystem.getInternal. FileSystem.getInternal also registers shutdown hook. In FileSystem of hadoop 0.23, the shutdown hook registration does not consider whether shutdown is in progress or not (In 2.2, it's considered). // 0.23 if (map.isEmpty() ) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } // 2.2 if (map.isEmpty() && !ShutdownHookManager.get().isShutdownInProgress()) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } Thus, in 0.23, another shutdown hook can be registered when ApplicationMaster's shutdown hook run. This issue cause IllegalStateException as follows. java.lang.IllegalStateException: Shutdown in progress, cannot add a shutdownHook at org.apache.hadoop.util.ShutdownHookManager.addShutdownHook(ShutdownHookManager.java:152) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2306) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2278) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:316) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:162) at org.apache.spark.deploy.yarn.ApplicationMaster.org$apache$spark$deploy$yarn$ApplicationMaster$$cleanupStagingDir(ApplicationMaster.scala:307) at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$3.run(ApplicationMaster.scala:118) at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54) Author: Kousuke Saruta Closes #2924 from sarutak/SPARK-3900-2 and squashes the following commits: 9112817 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900-2 97018fa [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 2c2850e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 ee52db2 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 a7d6c9b [Kousuke Saruta] Merge branch 'SPARK-3900' of github.com:sarutak/spark into SPARK-3900 1cdf03c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 a5f6443 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 57b397d [Kousuke Saruta] Fixed IllegalStateException caused by shutdown hook registration in another shutdown hook --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a3c43b43848d2..e6fe0265d8811 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -92,6 +92,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, logInfo("ApplicationAttemptId: " + appAttemptId) + val fs = FileSystem.get(yarnConf) val cleanupHook = new Runnable { override def run() { // If the SparkContext is still registered, shut it down as a best case effort in case @@ -115,7 +116,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // we only want to unregister if we don't want the RM to retry if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { unregister(finalStatus, finalMsg) - cleanupStagingDir() + cleanupStagingDir(fs) } } } @@ -303,8 +304,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, /** * Clean up the staging directory. */ - private def cleanupStagingDir() { - val fs = FileSystem.get(yarnConf) + private def cleanupStagingDir(fs: FileSystem) { var stagingDirPath: Path = null try { val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean From d60a9d440b00beb107c1f1d7f42886c94f04a092 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 24 Oct 2014 10:48:03 -0700 Subject: [PATCH 006/115] [SPARK-4051] [SQL] [PySpark] Convert Row into dictionary Added a method to Row to turn row into dict: ``` >>> row = Row(a=1) >>> row.asDict() {'a': 1} ``` Author: Davies Liu Closes #2896 from davies/dict and squashes the following commits: 8d97366 [Davies Liu] convert Row into dict --- python/pyspark/sql.py | 12 ++++++++++++ python/pyspark/tests.py | 9 +++++++++ 2 files changed, 21 insertions(+) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index b31a82f9b19ac..7daf306f68479 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -883,6 +883,10 @@ class Row(tuple): # create property for fast access locals().update(_create_properties(dataType.fields)) + def asDict(self): + """ Return as a dict """ + return dict(zip(self.__FIELDS__, self)) + def __repr__(self): # call collect __repr__ for nested objects return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) @@ -1466,6 +1470,14 @@ def __new__(self, *args, **kwargs): else: raise ValueError("No args or kwargs") + def asDict(self): + """ + Return as an dict + """ + if not hasattr(self, "__FIELDS__"): + raise TypeError("Cannot convert a Row class into dict") + return dict(zip(self.__FIELDS__, self)) + # let obect acs like class def __call__(self, *args): """create new Row object""" diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 7a2107ec326ee..047d85783089f 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -771,6 +771,15 @@ def test_serialize_nested_array_and_map(self): self.assertEqual(1.0, row.c) self.assertEqual("2", row.d) + def test_convert_row_to_dict(self): + row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) + self.assertEqual(1, row.asDict()['l'][0].a) + rdd = self.sc.parallelize([row]) + srdd = self.sqlCtx.inferSchema(rdd) + srdd.registerTempTable("test") + row = self.sqlCtx.sql("select l[0].a AS la from test").first() + self.assertEqual(1, row.asDict()["la"]) + class InputFormatTests(ReusedPySparkTestCase): From 0e886610eedd8ea24761cdcefa25ccedeca72dc8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 24 Oct 2014 10:52:25 -0700 Subject: [PATCH 007/115] [SPARK-4050][SQL] Fix caching of temporary tables with projections. Previously cached data was found by `sameResult` plan matching on optimized plans. This technique however fails to locate the cached data when a temporary table with a projection is queried with a further reduced projection. The failure is due to the fact that optimization will collapse the projections, producing a plan that no longer produces the sameResult as the cached data (though the cached data still subsumes the desired data). For example consider the following previously failing test case. ```scala sql("CACHE TABLE tempTable AS SELECT key FROM testData") assertCached(sql("SELECT COUNT(*) FROM tempTable")) ``` In this PR I change the matching to occur after analysis instead of optimization, so that in the case of temporary tables, the plans will always match. I think this should work generally, however, this error does raise questions about the need to do more thorough subsumption checking when locating cached data. Another question is what sort of semantics we want to provide when uncaching data from temporary tables. For example consider the following sequence of commands: ```scala testData.select('key).registerTempTable("tempTable1") testData.select('key).registerTempTable("tempTable2") cacheTable("tempTable1") // This obviously works. assertCached(sql("SELECT COUNT(*) FROM tempTable1")) // It seems good that this works ... assertCached(sql("SELECT COUNT(*) FROM tempTable2")) // ... but is this valid? uncacheTable("tempTable2") // Should this still be cached? assertCached(sql("SELECT COUNT(*) FROM tempTable1"), 0) ``` Author: Michael Armbrust Closes #2912 from marmbrus/cachingBug and squashes the following commits: 9c822d4 [Michael Armbrust] remove commented out code 5c72fb7 [Michael Armbrust] Add a test case / question about uncaching semantics. 63a23e4 [Michael Armbrust] Perform caching on analyzed instead of optimized plan. 03f1cfe [Michael Armbrust] Clean-up / add tests to SameResult suite. --- .../sql/catalyst/plans/SameResultSuite.scala | 10 ++++-- .../org/apache/spark/sql/CacheManager.scala | 10 +++--- .../org/apache/spark/sql/SQLContext.scala | 6 ++-- .../apache/spark/sql/CachedTableSuite.scala | 34 ++++++++++++++++++- 4 files changed, 48 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala index e8a793d107451..11e6831b24768 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ /** - * Provides helper methods for comparing plans. + * Tests for the sameResult function of [[LogicalPlan]]. */ class SameResultSuite extends FunSuite { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) @@ -52,11 +52,15 @@ class SameResultSuite extends FunSuite { assertSameResult(testRelation.select('a, 'b), testRelation2.select('a, 'b)) assertSameResult(testRelation.select('b, 'a), testRelation2.select('b, 'a)) - assertSameResult(testRelation, testRelation2.select('a), false) - assertSameResult(testRelation.select('b, 'a), testRelation2.select('a, 'b), false) + assertSameResult(testRelation, testRelation2.select('a), result = false) + assertSameResult(testRelation.select('b, 'a), testRelation2.select('a, 'b), result = false) } test("filters") { assertSameResult(testRelation.where('a === 'b), testRelation2.where('a === 'b)) } + + test("sorts") { + assertSameResult(testRelation.orderBy('a.asc), testRelation2.orderBy('a.asc)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index 5ab2b5316ab10..3ced11a5e6c11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -82,7 +82,7 @@ private[sql] trait CacheManager { private[sql] def cacheQuery( query: SchemaRDD, storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock { - val planToCache = query.queryExecution.optimizedPlan + val planToCache = query.queryExecution.analyzed if (lookupCachedData(planToCache).nonEmpty) { logWarning("Asked to cache already cached data.") } else { @@ -96,8 +96,8 @@ private[sql] trait CacheManager { /** Removes the data for the given SchemaRDD from the cache */ private[sql] def uncacheQuery(query: SchemaRDD, blocking: Boolean = true): Unit = writeLock { - val planToCache = query.queryExecution.optimizedPlan - val dataIndex = cachedData.indexWhere(_.plan.sameResult(planToCache)) + val planToCache = query.queryExecution.analyzed + val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) require(dataIndex >= 0, s"Table $query is not cached.") cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) cachedData.remove(dataIndex) @@ -106,12 +106,12 @@ private[sql] trait CacheManager { /** Optionally returns cached data for the given SchemaRDD */ private[sql] def lookupCachedData(query: SchemaRDD): Option[CachedData] = readLock { - lookupCachedData(query.queryExecution.optimizedPlan) + lookupCachedData(query.queryExecution.analyzed) } /** Optionally returns cached data for the given LogicalPlan. */ private[sql] def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { - cachedData.find(_.plan.sameResult(plan)) + cachedData.find(cd => plan.sameResult(cd.plan)) } /** Replaces segments of the given logical plan with cached versions where possible. */ 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 0e4a9ca60b00d..590dbf3cb893d 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 @@ -374,13 +374,13 @@ class SQLContext(@transient val sparkContext: SparkContext) def logical: LogicalPlan lazy val analyzed = ExtractPythonUdfs(analyzer(logical)) - lazy val optimizedPlan = optimizer(analyzed) - lazy val withCachedData = useCachedData(optimizedPlan) + lazy val withCachedData = useCachedData(analyzed) + lazy val optimizedPlan = optimizer(withCachedData) // TODO: Don't just pick the first one... lazy val sparkPlan = { SparkPlan.currentContext.set(self) - planner(withCachedData).next() + planner(optimizedPlan).next() } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 444bc95009c31..da5a358df3b1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -53,17 +53,47 @@ class CachedTableSuite extends QueryTest { sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty } + test("cache temp table") { + testData.select('key).registerTempTable("tempTable") + assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) + cacheTable("tempTable") + assertCached(sql("SELECT COUNT(*) FROM tempTable")) + uncacheTable("tempTable") + } + + test("cache table as select") { + sql("CACHE TABLE tempTable AS SELECT key FROM testData") + assertCached(sql("SELECT COUNT(*) FROM tempTable")) + uncacheTable("tempTable") + } + + test("uncaching temp table") { + testData.select('key).registerTempTable("tempTable1") + testData.select('key).registerTempTable("tempTable2") + cacheTable("tempTable1") + + assertCached(sql("SELECT COUNT(*) FROM tempTable1")) + assertCached(sql("SELECT COUNT(*) FROM tempTable2")) + + // Is this valid? + uncacheTable("tempTable2") + + // Should this be cached? + assertCached(sql("SELECT COUNT(*) FROM tempTable1"), 0) + } + test("too big for memory") { val data = "*" * 10000 sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).registerTempTable("bigData") table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(table("bigData").count() === 200000L) - table("bigData").unpersist() + table("bigData").unpersist(blocking = true) } test("calling .cache() should use in-memory columnar caching") { table("testData").cache() assertCached(table("testData")) + table("testData").unpersist(blocking = true) } test("calling .unpersist() should drop in-memory columnar cache") { @@ -108,6 +138,8 @@ class CachedTableSuite extends QueryTest { case r @ InMemoryRelation(_, _, _, _, _: InMemoryColumnarTableScan) => r }.size } + + uncacheTable("testData") } test("read from cached table and uncache") { From 7c89a8f0c81ecf91dba34c1f44393f45845d438c Mon Sep 17 00:00:00 2001 From: Zhan Zhang Date: Fri, 24 Oct 2014 11:03:17 -0700 Subject: [PATCH 008/115] [SPARK-2706][SQL] Enable Spark to support Hive 0.13 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Given that a lot of users are trying to use hive 0.13 in spark, and the incompatibility between hive-0.12 and hive-0.13 on the API level I want to propose following approach, which has no or minimum impact on existing hive-0.12 support, but be able to jumpstart the development of hive-0.13 and future version support. Approach: Introduce “hive-version” property, and manipulate pom.xml files to support different hive version at compiling time through shim layer, e.g., hive-0.12.0 and hive-0.13.1. More specifically, 1. For each different hive version, there is a very light layer of shim code to handle API differences, sitting in sql/hive/hive-version, e.g., sql/hive/v0.12.0 or sql/hive/v0.13.1 2. Add a new profile hive-default active by default, which picks up all existing configuration and hive-0.12.0 shim (v0.12.0) if no hive.version is specified. 3. If user specifies different version (currently only 0.13.1 by -Dhive.version = 0.13.1), hive-versions profile will be activated, which pick up hive-version specific shim layer and configuration, mainly the hive jars and hive-version shim, e.g., v0.13.1. 4. With this approach, nothing is changed with current hive-0.12 support. No change by default: sbt/sbt -Phive For example: sbt/sbt -Phive -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 assembly To enable hive-0.13: sbt/sbt -Dhive.version=0.13.1 For example: sbt/sbt -Dhive.version=0.13.1 -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 assembly Note that in hive-0.13, hive-thriftserver is not enabled, which should be fixed by other Jira, and we don’t need -Phive with -Dhive.version in building (probably we should use -Phive -Dhive.version=xxx instead after thrift server is also supported in hive-0.13.1). Author: Zhan Zhang Author: zhzhan Author: Patrick Wendell Closes #2241 from zhzhan/spark-2706 and squashes the following commits: 3ece905 [Zhan Zhang] minor fix 410b668 [Zhan Zhang] solve review comments cbb4691 [Zhan Zhang] change run-test for new options 0d4d2ed [Zhan Zhang] rebase 497b0f4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 8fad1cf [Zhan Zhang] change the pom file and make hive-0.13.1 as the default ab028d1 [Zhan Zhang] rebase 4a2e36d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 4cb1b93 [zhzhan] Merge pull request #1 from pwendell/pr-2241 b0478c0 [Patrick Wendell] Changes to simplify the build of SPARK-2706 2b50502 [Zhan Zhang] rebase a72c0d4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark cb22863 [Zhan Zhang] correct the typo 20f6cf7 [Zhan Zhang] solve compatability issue f7912a9 [Zhan Zhang] rebase and solve review feedback 301eb4a [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 10c3565 [Zhan Zhang] address review comments 6bc9204 [Zhan Zhang] rebase and remove temparory repo d3aa3f2 [Zhan Zhang] Merge branch 'master' into spark-2706 cedcc6f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 3ced0d7 [Zhan Zhang] rebase d9b981d [Zhan Zhang] rebase and fix error due to rollback adf4924 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 3dd50e8 [Zhan Zhang] solve conflicts and remove unnecessary implicts d10bf00 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark dc7bdb3 [Zhan Zhang] solve conflicts 7e0cc36 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark d7c3e1e [Zhan Zhang] Merge branch 'master' into spark-2706 68deb11 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark d48bd18 [Zhan Zhang] address review comments 3ee3b2b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 57ea52e [Zhan Zhang] Merge branch 'master' into spark-2706 2b0d513 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 9412d24 [Zhan Zhang] address review comments f4af934 [Zhan Zhang] rebase 1ccd7cc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 128b60b [Zhan Zhang] ignore 0.12.0 test cases for the time being af9feb9 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 5f5619f [Zhan Zhang] restructure the directory and different hive version support 05d3683 [Zhan Zhang] solve conflicts e4c1982 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 94b4fdc [Zhan Zhang] Spark-2706: hive-0.13.1 support on spark 87ebf3b [Zhan Zhang] Merge branch 'master' into spark-2706 921e914 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark f896b2a [Zhan Zhang] Merge branch 'master' into spark-2706 789ea21 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark cb53a2c [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark f6a8a40 [Zhan Zhang] revert ba14f28 [Zhan Zhang] test dbedff3 [Zhan Zhang] Merge remote-tracking branch 'upstream/master' 70964fe [Zhan Zhang] revert fe0f379 [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark 70ffd93 [Zhan Zhang] revert 42585ec [Zhan Zhang] test 7d5fce2 [Zhan Zhang] test --- assembly/pom.xml | 6 + dev/run-tests | 4 +- docs/building-spark.md | 26 ++- pom.xml | 29 ++- sql/hive/pom.xml | 37 +++- .../apache/spark/sql/hive/HiveContext.scala | 23 ++- .../spark/sql/hive/HiveInspectors.scala | 3 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 10 +- .../org/apache/spark/sql/hive/HiveQl.scala | 16 +- .../apache/spark/sql/hive/TableReader.scala | 3 +- .../org/apache/spark/sql/hive/TestHive.scala | 5 + .../execution/DescribeHiveTableCommand.scala | 4 +- .../sql/hive/execution/HiveTableScan.scala | 4 +- .../hive/execution/InsertIntoHiveTable.scala | 8 +- .../spark/sql/hive/hiveWriterContainers.scala | 3 +- .../spark/sql/hive/StatisticsSuite.scala | 7 +- .../sql/hive/execution/HiveQuerySuite.scala | 22 ++- .../org/apache/spark/sql/hive/Shim.scala | 89 +++++++++ .../org/apache/spark/sql/hive/Shim.scala | 170 ++++++++++++++++++ 19 files changed, 406 insertions(+), 63 deletions(-) create mode 100644 sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala create mode 100644 sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala diff --git a/assembly/pom.xml b/assembly/pom.xml index 31a01e4d8e1de..bfef95b8deb95 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -197,6 +197,12 @@ spark-hive_${scala.binary.version} ${project.version} + + + + + hive-0.12.0 + org.apache.spark spark-hive-thriftserver_${scala.binary.version} diff --git a/dev/run-tests b/dev/run-tests index f47fcf66ff7e7..7d06c86eb4b41 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -140,7 +140,7 @@ CURRENT_BLOCK=$BLOCK_BUILD { # We always build with Hive because the PySpark Spark SQL tests need it. - BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" echo "[info] Building Spark with these arguments: $BUILD_MVN_PROFILE_ARGS" @@ -167,7 +167,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. # This must be a single argument, as it is. if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" fi if [ -n "$_SQL_TESTS_ONLY" ]; then diff --git a/docs/building-spark.md b/docs/building-spark.md index b2940ee4029e8..11fd56c145c01 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -97,12 +97,20 @@ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package {% endhighlight %} + + # Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, -add the `-Phive` profile to your existing build options. +add the `-Phive` profile to your existing build options. By default Spark +will build with Hive 0.13.1 bindings. You can also build for Hive 0.12.0 using +the `-Phive-0.12.0` profile. NOTE: currently the JDBC server is only +supported for Hive 0.12.0. {% highlight bash %} -# Apache Hadoop 2.4.X with Hive support +# Apache Hadoop 2.4.X with Hive 13 support mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package + +# Apache Hadoop 2.4.X with Hive 12 support +mvn -Pyarn -Phive-0.12.0 -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package {% endhighlight %} # Spark Tests in Maven @@ -111,8 +119,8 @@ Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.o Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: - mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive clean package - mvn -Pyarn -Phadoop-2.3 -Phive test + mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-0.12.0 clean package + mvn -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 test The ScalaTest plugin also supports running only a specific test suite as follows: @@ -175,16 +183,16 @@ can be set to control the SBT build. For example: Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive assembly - sbt/sbt -Pyarn -Phadoop-2.3 -Phive test + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 assembly + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 test To run only a specific test suite as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive "test-only org.apache.spark.repl.ReplSuite" + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 "test-only org.apache.spark.repl.ReplSuite" To run test suites of a specific sub project as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive core/test + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 core/test # Speeding up Compilation with Zinc @@ -192,4 +200,4 @@ To run test suites of a specific sub project as follows: compiler. When run locally as a background process, it speeds up builds of Scala-based projects like Spark. Developers who regularly recompile Spark with Maven will be the most interested in Zinc. The project site gives instructions for building and running `zinc`; OS X users can -install it using `brew install zinc`. \ No newline at end of file +install it using `brew install zinc`. diff --git a/pom.xml b/pom.xml index a9897b866b036..a1195262614dd 100644 --- a/pom.xml +++ b/pom.xml @@ -127,7 +127,11 @@ 0.94.6 1.4.0 3.4.5 - 0.12.0-protobuf-2.5 + + 0.13.1 + + 0.13.1 + 10.10.1.1 1.4.3 1.2.3 8.1.14.v20131031 @@ -456,7 +460,7 @@ org.apache.derby derby - 10.4.2.0 + ${derby.version} com.codahale.metrics @@ -1308,16 +1312,31 @@ - - hive + hive-0.12.0 false + sql/hive-thriftserver + + 0.12.0-protobuf-2.5 + 0.12.0 + 10.4.2.0 + + + + hive-0.13.1 + + false + + + 0.13.1 + 0.13.1 + 10.10.1.1 + - diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 9d7a02bf7b0b7..db01363b4d629 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -36,11 +36,6 @@ - - com.twitter - parquet-hive-bundle - 1.5.0 - org.apache.spark spark-core_${scala.binary.version} @@ -116,7 +111,6 @@ test - hive @@ -144,6 +138,19 @@ + + hive-0.12.0 + + false + + + + com.twitter + parquet-hive-bundle + 1.5.0 + + + @@ -154,6 +161,24 @@ org.scalatest scalatest-maven-plugin + + org.codehaus.mojo + build-helper-maven-plugin + + + add-default-sources + generate-sources + + add-source + + + + v${hive.version.short}/src/main/scala + + + + + 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 8b5a90159e1bb..34ed57b001637 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 @@ -32,7 +32,6 @@ import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.io.TimestampWritable import org.apache.hadoop.hive.serde2.io.DateWritable @@ -47,6 +46,7 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand +import org.apache.spark.sql.hive.HiveShim /** * DEPRECATED: Use HiveContext instead. @@ -171,13 +171,15 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val tableParameters = relation.hiveQlTable.getParameters val oldTotalSize = - Option(tableParameters.get(StatsSetupConst.TOTAL_SIZE)).map(_.toLong).getOrElse(0L) + Option(tableParameters.get(HiveShim.getStatsSetupConstTotalSize)) + .map(_.toLong) + .getOrElse(0L) val newTotalSize = getFileSizeForTable(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(). if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - tableParameters.put(StatsSetupConst.TOTAL_SIZE, newTotalSize.toString) + tableParameters.put(HiveShim.getStatsSetupConstTotalSize, newTotalSize.toString) val hiveTTable = relation.hiveQlTable.getTTable hiveTTable.setParameters(tableParameters) val tableFullName = @@ -282,29 +284,24 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { */ protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { try { - // Session state must be initilized before the CommandProcessor is created . - SessionState.start(sessionState) - val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) + val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hiveconf) proc match { case driver: Driver => - driver.init() - - val results = new JArrayList[String] + val results = HiveShim.createDriverResultsArray val response: CommandProcessorResponse = driver.run(cmd) // Throw an exception if there is an error in query processing. if (response.getResponseCode != 0) { - driver.destroy() + driver.close() throw new QueryExecutionException(response.getErrorMessage) } driver.setMaxRows(maxRows) driver.getResults(results) - driver.destroy() - results + driver.close() + HiveShim.processResults(results) case _ => sessionState.out.println(tokens(0) + " " + cmd_1) Seq(proc.run(cmd_1).getResponseCode.toString) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 1977618b4c9f2..deaa1a2a154f2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.hive.HiveShim /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -149,7 +150,7 @@ private[hive] trait HiveInspectors { case l: Long => l: java.lang.Long case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte - case b: BigDecimal => new HiveDecimal(b.underlying()) + case b: BigDecimal => HiveShim.createDecimal(b.underlying()) case b: Array[Byte] => b case d: java.sql.Date => d case t: java.sql.Timestamp => t 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 75a19656af110..904bb48691e35 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 @@ -22,7 +22,6 @@ import scala.util.parsing.combinator.RegexParsers import org.apache.hadoop.hive.metastore.api.{FieldSchema, SerDeInfo, StorageDescriptor, Partition => TPartition, Table => TTable} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.Logging @@ -34,6 +33,7 @@ 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.catalyst.types._ +import org.apache.spark.sql.hive.HiveShim import org.apache.spark.util.Utils /* Implicit conversions */ @@ -56,7 +56,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val table = client.getTable(databaseName, tblName) val partitions: Seq[Partition] = if (table.isPartitioned) { - client.getAllPartitionsForPruner(table).toSeq + HiveShim.getAllPartitionsOf(client, table).toSeq } else { Nil } @@ -185,7 +185,7 @@ object HiveMetastoreTypes extends RegexParsers { "bigint" ^^^ LongType | "binary" ^^^ BinaryType | "boolean" ^^^ BooleanType | - "decimal" ^^^ DecimalType | + HiveShim.metastoreDecimal ^^^ DecimalType | "date" ^^^ DateType | "timestamp" ^^^ TimestampType | "varchar\\((\\d+)\\)".r ^^^ StringType @@ -272,13 +272,13 @@ private[hive] case class MetastoreRelation // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, // `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future. BigInt( - Option(hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE)) + Option(hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize)) .map(_.toLong) .getOrElse(sqlContext.defaultSizeInBytes)) } ) - val tableDesc = new TableDesc( + val tableDesc = HiveShim.getTableDesc( Class.forName( hiveQlTable.getSerializationLib, true, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 2b599157d15d3..ffcb6b505b9c6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.hive import java.sql.Date - +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan.PlanUtils @@ -216,7 +217,18 @@ private[hive] object HiveQl { /** * Returns the AST for the given SQL string. */ - def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql)) + def getAst(sql: String): ASTNode = { + /* + * Context has to be passed in hive0.13.1. + * Otherwise, there will be Null pointer exception, + * when retrieving properties form HiveConf. + */ + val hContext = new Context(new HiveConf()) + val node = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql, hContext)) + hContext.clear() + node + } + /** Returns a LogicalPlan for a given HiveQL string. */ def parseSql(sql: String): LogicalPlan = hqlParser(sql) 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 fd4f65e488259..e45eb57b3debf 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 @@ -34,6 +34,7 @@ import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.hive.HiveShim /** * A trait for subclasses that handle table scans. @@ -138,7 +139,7 @@ class HadoopTableReader( filterOpt: Option[PathFilter]): RDD[Row] = { val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) - val partPath = partition.getPartitionPath + val partPath = HiveShim.getDataLocationPath(partition) val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) val ifc = partDesc.getInputFileFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 9a9e2eda6bcd4..0f74fe8943706 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -26,6 +26,7 @@ import scala.language.implicitConversions import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} import org.apache.hadoop.hive.ql.metadata.Table +import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.RegexSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.avro.AvroSerDe @@ -63,6 +64,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { // 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) lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath @@ -375,6 +377,9 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { */ protected val originalUdfs: JavaSet[String] = FunctionRegistry.getFunctionNames + // Database default may not exist in 0.13.1, create it if not exist + HiveShim.createDefaultDBIfNeeded(this) + /** * Resets the test instance by deleting any tables that have been created. * TODO: also clear out UDFs, views, etc. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index 106cede9788ec..fbd375639692f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.execution.{Command, LeafNode} import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} +import org.apache.spark.sql.hive.HiveShim /** * Implementation for "describe [extended] table". @@ -43,7 +44,8 @@ case class DescribeHiveTableCommand( // Strings with the format like Hive. It is used for result comparison in our unit tests. lazy val hiveString: Seq[String] = sideEffectResult.map { case Row(name: String, dataType: String, comment) => - Seq(name, dataType, Option(comment.asInstanceOf[String]).getOrElse("None")) + Seq(name, dataType, + Option(comment.asInstanceOf[String]).getOrElse(HiveShim.getEmptyCommentsFieldValue)) .map(s => String.format(s"%-20s", s)) .mkString("\t") } 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 5b83b77d80a22..85965a6ea095a 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 @@ -23,7 +23,6 @@ import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.objectinspector.primitive._ @@ -83,8 +82,7 @@ case class HiveTableScan( attributes.map(a => relation.attributes.indexWhere(_.name == a.name): Integer).filter(index => index >= 0) - ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) - ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) + HiveShim.appendReadColumns(hiveConf, neededColumnIDs, attributes.map(_.name)) val tableDesc = relation.tableDesc val deserializer = tableDesc.getDeserializerClass.newInstance 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 f0785d8882636..7db5fd804d6ef 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 @@ -24,7 +24,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.{Context, ErrorMsg} import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption @@ -37,6 +37,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode} import org.apache.spark.sql.hive._ +import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.{SerializableWritable, SparkException, TaskContext} /** @@ -74,7 +76,7 @@ case class InsertIntoHiveTable( (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) case _: JavaHiveDecimalObjectInspector => - (o: Any) => new HiveDecimal(o.asInstanceOf[BigDecimal].underlying()) + (o: Any) => HiveShim.createDecimal(o.asInstanceOf[BigDecimal].underlying()) case soi: StandardStructObjectInspector => val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) @@ -170,7 +172,7 @@ case class InsertIntoHiveTable( // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc val tableLocation = table.hiveQlTable.getDataLocation - val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) + val tmpLocation = HiveShim.getExternalTmpPath(hiveContext, tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) val isCompressed = sc.hiveconf.getBoolean( ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 6ccbc22a4acfb..981ab954da489 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -27,12 +27,13 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.FileSinkDesc import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ import org.apache.spark.sql.Row import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} +import org.apache.spark.sql.hive.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.hive.HiveShim._ /** * Internal helper class that saves an RDD using a Hive OutputFormat. 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 14e791fe0f0ee..aaefe84ce81ea 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 @@ -25,6 +25,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.NativeCommand import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -80,8 +81,10 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() - assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) - + // TODO: How does it works? needs to add it back for other hive version. + if (HiveShim.version =="0.12.0") { + assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) + } sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") assert(queryTotalSize("analyzeTable") === BigInt(11624)) 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 3e100775e4981..5de20175d9f57 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 @@ -508,14 +508,14 @@ class HiveQuerySuite extends HiveComparisonTest { // Describe a partition is a native command assertResult( Array( - Array("key", "int", "None"), - Array("value", "string", "None"), - Array("dt", "string", "None"), + Array("key", "int", HiveShim.getEmptyCommentsFieldValue), + Array("value", "string", HiveShim.getEmptyCommentsFieldValue), + Array("dt", "string", HiveShim.getEmptyCommentsFieldValue), Array("", "", ""), Array("# Partition Information", "", ""), Array("# col_name", "data_type", "comment"), Array("", "", ""), - Array("dt", "string", "None")) + Array("dt", "string", HiveShim.getEmptyCommentsFieldValue)) ) { sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") .select('result) @@ -561,11 +561,15 @@ class HiveQuerySuite extends HiveComparisonTest { |WITH serdeproperties('s1'='9') """.stripMargin) } - sql(s"ADD JAR $testJar") - sql( - """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' - |WITH serdeproperties('s1'='9') - """.stripMargin) + // Now only verify 0.12.0, and ignore other versions due to binary compatability + // current TestSerDe.jar is from 0.12.0 + if (HiveShim.version == "0.12.0") { + sql(s"ADD JAR $testJar") + sql( + """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' + |WITH serdeproperties('s1'='9') + """.stripMargin) + } sql("DROP TABLE alter1") } diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala new file mode 100644 index 0000000000000..6dde636965afd --- /dev/null +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.net.URI +import java.util.{ArrayList => JArrayList} +import java.util.Properties +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.processors._ +import org.apache.hadoop.hive.ql.stats.StatsSetupConst +import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} +import org.apache.hadoop.{io => hadoopIo} +import org.apache.hadoop.mapred.InputFormat +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +/** + * A compatibility layer for interacting with Hive version 0.12.0. + */ +private[hive] object HiveShim { + val version = "0.12.0" + val metastoreDecimal = "decimal" + + def getTableDesc( + serdeClass: Class[_ <: Deserializer], + inputFormatClass: Class[_ <: InputFormat[_, _]], + outputFormatClass: Class[_], + properties: Properties) = { + new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) + } + + def createDriverResultsArray = new JArrayList[String] + + def processResults(results: JArrayList[String]) = results + + def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + + def createDefaultDBIfNeeded(context: HiveContext) = { } + + /** The string used to denote an empty comments field in the schema. */ + def getEmptyCommentsFieldValue = "None" + + def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { + CommandProcessorFactory.get(cmd(0), conf) + } + + def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { + new HiveDecimal(bd) + } + + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + ColumnProjectionUtils.appendReadColumnIDs(conf, ids) + ColumnProjectionUtils.appendReadColumnNames(conf, names) + } + + def getExternalTmpPath(context: Context, uri: URI) = { + context.getExternalTmpFileURI(uri) + } + + def getDataLocationPath(p: Partition) = p.getPartitionPath + + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) + +} + +class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) + extends FileSinkDesc(dir, tableInfo, compressed) { +} diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala new file mode 100644 index 0000000000000..8678c0c475db4 --- /dev/null +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.util.{ArrayList => JArrayList} +import java.util.Properties +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.StatsSetupConst +import org.apache.hadoop.hive.common.`type`.{HiveDecimal} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} +import org.apache.hadoop.mapred.InputFormat +import org.apache.spark.Logging +import org.apache.hadoop.{io => hadoopIo} +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +/** + * A compatibility layer for interacting with Hive version 0.13.1. + */ +private[hive] object HiveShim { + val version = "0.13.1" + /* + * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded) + * Full support of new decimal feature need to be fixed in seperate PR. + */ + val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r + + def getTableDesc( + serdeClass: Class[_ <: Deserializer], + inputFormatClass: Class[_ <: InputFormat[_, _]], + outputFormatClass: Class[_], + properties: Properties) = { + new TableDesc(inputFormatClass, outputFormatClass, properties) + } + + def createDriverResultsArray = new JArrayList[Object] + + def processResults(results: JArrayList[Object]) = { + results.map { r => + r match { + case s: String => s + case a: Array[Object] => a(0).asInstanceOf[String] + } + } + } + + def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + + def createDefaultDBIfNeeded(context: HiveContext) = { + context.runSqlHive("CREATE DATABASE default") + context.runSqlHive("USE default") + } + + /* The string used to denote an empty comments field in the schema. */ + def getEmptyCommentsFieldValue = "" + + def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { + CommandProcessorFactory.get(cmd, conf) + } + + def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { + HiveDecimal.create(bd) + } + + /* + * This function in hive-0.13 become private, but we have to do this to walkaround hive bug + */ + private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) { + val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "") + val result: StringBuilder = new StringBuilder(old) + var first: Boolean = old.isEmpty + + for (col <- cols) { + if (first) { + first = false + } else { + result.append(',') + } + result.append(col) + } + conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString) + } + + /* + * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty + */ + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + if (ids != null && ids.size > 0) { + ColumnProjectionUtils.appendReadColumns(conf, ids) + } + if (names != null && names.size > 0) { + appendReadColumnNames(conf, names) + } + } + + def getExternalTmpPath(context: Context, path: Path) = { + context.getExternalTmpPath(path.toUri) + } + + def getDataLocationPath(p: Partition) = p.getDataLocation + + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsOf(tbl) + + /* + * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not. + * Fix it through wrapper. + * */ + implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = { + var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed) + f.setCompressCodec(w.compressCodec) + f.setCompressType(w.compressType) + f.setTableInfo(w.tableInfo) + f.setDestTableId(w.destTableId) + f + } +} + +/* + * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. + * Fix it through wrapper. + */ +class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) + extends Serializable with Logging { + var compressCodec: String = _ + var compressType: String = _ + var destTableId: Int = _ + + def setCompressed(compressed: Boolean) { + this.compressed = compressed + } + + def getDirName = dir + + def setDestTableId(destTableId: Int) { + this.destTableId = destTableId + } + + def setTableInfo(tableInfo: TableDesc) { + this.tableInfo = tableInfo + } + + def setCompressCodec(intermediateCompressorCodec: String) { + compressCodec = intermediateCompressorCodec + } + + def setCompressType(intermediateCompressType: String) { + compressType = intermediateCompressType + } +} From 6a40a76848203d7266c134a26191579138c76903 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 24 Oct 2014 11:44:48 -0700 Subject: [PATCH 009/115] [SPARK-4026][Streaming] Write ahead log management As part of the effort to avoid data loss on Spark Streaming driver failure, we want to implement a write ahead log that can write received data to HDFS. This allows the received data to be persist across driver failures. So when the streaming driver is restarted, it can find and reprocess all the data that were received but not processed. This was primarily implemented by @harishreedharan. This is still WIP, as he is going to improve the unitests by using HDFS mini cluster. Author: Hari Shreedharan Author: Tathagata Das Closes #2882 from tdas/driver-ha-wal and squashes the following commits: e4bee20 [Tathagata Das] Removed synchronized, Path.getFileSystem is threadsafe 55514e2 [Tathagata Das] Minor changes based on PR comments. d29fddd [Tathagata Das] Merge pull request #20 from harishreedharan/driver-ha-wal a317a4d [Hari Shreedharan] Directory deletion should not fail tests 9514dc8 [Tathagata Das] Added unit tests to test reading of corrupted data and other minor edits 3881706 [Tathagata Das] Merge pull request #19 from harishreedharan/driver-ha-wal 4705fff [Hari Shreedharan] Sort listed files by name. Use local files for WAL tests. eb356ca [Tathagata Das] Merge pull request #18 from harishreedharan/driver-ha-wal 82ce56e [Hari Shreedharan] Fix file ordering issue in WALManager tests 5ff90ee [Hari Shreedharan] Fix tests to not ignore ordering and also assert all data is present ef8db09 [Tathagata Das] Merge pull request #17 from harishreedharan/driver-ha-wal 7e40e56 [Hari Shreedharan] Restore old build directory after tests 587b876 [Hari Shreedharan] Fix broken test. Call getFileSystem only from synchronized method. b4be0c1 [Hari Shreedharan] Remove unused method edcbee1 [Hari Shreedharan] Tests reading and writing data using writers now use Minicluster. 5c70d1f [Hari Shreedharan] Remove underlying stream from the WALWriter. 4ab602a [Tathagata Das] Refactored write ahead stuff from streaming.storage to streaming.util b06be2b [Tathagata Das] Adding missing license. 5182ffb [Hari Shreedharan] Added documentation 172358d [Tathagata Das] Pulled WriteAheadLog-related stuff from tdas/spark/tree/driver-ha-working --- .../spark/streaming/util/HdfsUtils.scala | 72 ++++ .../util/WriteAheadLogFileSegment.scala | 20 + .../streaming/util/WriteAheadLogManager.scala | 224 +++++++++++ .../util/WriteAheadLogRandomReader.scala | 55 +++ .../streaming/util/WriteAheadLogReader.scala | 82 ++++ .../streaming/util/WriteAheadLogWriter.scala | 82 ++++ .../streaming/util/WriteAheadLogSuite.scala | 357 ++++++++++++++++++ 7 files changed, 892 insertions(+) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala new file mode 100644 index 0000000000000..491f1175576e6 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ + +private[streaming] object HdfsUtils { + + def getOutputStream(path: String, conf: Configuration): FSDataOutputStream = { + val dfsPath = new Path(path) + val dfs = getFileSystemForPath(dfsPath, conf) + // If the file exists and we have append support, append instead of creating a new file + val stream: FSDataOutputStream = { + if (dfs.isFile(dfsPath)) { + if (conf.getBoolean("hdfs.append.support", false)) { + dfs.append(dfsPath) + } else { + throw new IllegalStateException("File exists and there is no append support!") + } + } else { + dfs.create(dfsPath) + } + } + stream + } + + def getInputStream(path: String, conf: Configuration): FSDataInputStream = { + val dfsPath = new Path(path) + val dfs = getFileSystemForPath(dfsPath, conf) + val instream = dfs.open(dfsPath) + instream + } + + def checkState(state: Boolean, errorMsg: => String) { + if (!state) { + throw new IllegalStateException(errorMsg) + } + } + + def getBlockLocations(path: String, conf: Configuration): Option[Array[String]] = { + val dfsPath = new Path(path) + val dfs = getFileSystemForPath(dfsPath, conf) + val fileStatus = dfs.getFileStatus(dfsPath) + val blockLocs = Option(dfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen)) + blockLocs.map(_.flatMap(_.getHosts)) + } + + def getFileSystemForPath(path: Path, conf: Configuration): FileSystem = { + // For local file systems, return the raw loca file system, such calls to flush() + // actually flushes the stream. + val fs = path.getFileSystem(conf) + fs match { + case localFs: LocalFileSystem => localFs.getRawFileSystem + case _ => fs + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala new file mode 100644 index 0000000000000..1005a2c8ec303 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +/** Class for representing a segment of data in a write ahead log file */ +private[streaming] case class WriteAheadLogFileSegment (path: String, offset: Long, length: Int) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala new file mode 100644 index 0000000000000..70d234320be7c --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.{ExecutionContext, Future} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.permission.FsPermission +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import WriteAheadLogManager._ + +/** + * This class manages write ahead log files. + * - Writes records (bytebuffers) to periodically rotating log files. + * - Recovers the log files and the reads the recovered records upon failures. + * - Cleans up old log files. + * + * Uses [[org.apache.spark.streaming.util.WriteAheadLogWriter]] to write + * and [[org.apache.spark.streaming.util.WriteAheadLogReader]] to read. + * + * @param logDirectory Directory when rotating log files will be created. + * @param hadoopConf Hadoop configuration for reading/writing log files. + * @param rollingIntervalSecs The interval in seconds with which logs will be rolled over. + * Default is one minute. + * @param maxFailures Max number of failures that is tolerated for every attempt to write to log. + * Default is three. + * @param callerName Optional name of the class who is using this manager. + * @param clock Optional clock that is used to check for rotation interval. + */ +private[streaming] class WriteAheadLogManager( + logDirectory: String, + hadoopConf: Configuration, + rollingIntervalSecs: Int = 60, + maxFailures: Int = 3, + callerName: String = "", + clock: Clock = new SystemClock + ) extends Logging { + + private val pastLogs = new ArrayBuffer[LogInfo] + private val callerNameTag = + if (callerName.nonEmpty) s" for $callerName" else "" + private val threadpoolName = s"WriteAheadLogManager $callerNameTag" + implicit private val executionContext = ExecutionContext.fromExecutorService( + Utils.newDaemonFixedThreadPool(1, threadpoolName)) + override protected val logName = s"WriteAheadLogManager $callerNameTag" + + private var currentLogPath: Option[String] = None + private var currentLogWriter: WriteAheadLogWriter = null + private var currentLogWriterStartTime: Long = -1L + private var currentLogWriterStopTime: Long = -1L + + initializeOrRecover() + + /** + * Write a byte buffer to the log file. This method synchronously writes the data in the + * ByteBuffer to HDFS. When this method returns, the data is guaranteed to have been flushed + * to HDFS, and will be available for readers to read. + */ + def writeToLog(byteBuffer: ByteBuffer): WriteAheadLogFileSegment = synchronized { + var fileSegment: WriteAheadLogFileSegment = null + var failures = 0 + var lastException: Exception = null + var succeeded = false + while (!succeeded && failures < maxFailures) { + try { + fileSegment = getLogWriter(clock.currentTime).write(byteBuffer) + succeeded = true + } catch { + case ex: Exception => + lastException = ex + logWarning("Failed to write to write ahead log") + resetWriter() + failures += 1 + } + } + if (fileSegment == null) { + logError(s"Failed to write to write ahead log after $failures failures") + throw lastException + } + fileSegment + } + + /** + * Read all the existing logs from the log directory. + * + * Note that this is typically called when the caller is initializing and wants + * to recover past state from the write ahead logs (that is, before making any writes). + * If this is called after writes have been made using this manager, then it may not return + * the latest the records. This does not deal with currently active log files, and + * hence the implementation is kept simple. + */ + def readFromLog(): Iterator[ByteBuffer] = synchronized { + val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath + logInfo("Reading from the logs: " + logFilesToRead.mkString("\n")) + logFilesToRead.iterator.map { file => + logDebug(s"Creating log reader with $file") + new WriteAheadLogReader(file, hadoopConf) + } flatMap { x => x } + } + + /** + * Delete the log files that are older than the threshold time. + * + * Its important to note that the threshold time is based on the time stamps used in the log + * files, which is usually based on the local system time. So if there is coordination necessary + * between the node calculating the threshTime (say, driver node), and the local system time + * (say, worker node), the caller has to take account of possible time skew. + */ + def cleanupOldLogs(threshTime: Long): Unit = { + val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } } + logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + + s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") + + def deleteFiles() { + oldLogFiles.foreach { logInfo => + try { + val path = new Path(logInfo.path) + val fs = HdfsUtils.getFileSystemForPath(path, hadoopConf) + fs.delete(path, true) + synchronized { pastLogs -= logInfo } + logDebug(s"Cleared log file $logInfo") + } catch { + case ex: Exception => + logWarning(s"Error clearing write ahead log file $logInfo", ex) + } + } + logInfo(s"Cleared log files in $logDirectory older than $threshTime") + } + if (!executionContext.isShutdown) { + Future { deleteFiles() } + } + } + + /** Stop the manager, close any open log writer */ + def stop(): Unit = synchronized { + if (currentLogWriter != null) { + currentLogWriter.close() + } + executionContext.shutdown() + logInfo("Stopped write ahead log manager") + } + + /** Get the current log writer while taking care of rotation */ + private def getLogWriter(currentTime: Long): WriteAheadLogWriter = synchronized { + if (currentLogWriter == null || currentTime > currentLogWriterStopTime) { + resetWriter() + currentLogPath.foreach { + pastLogs += LogInfo(currentLogWriterStartTime, currentLogWriterStopTime, _) + } + currentLogWriterStartTime = currentTime + currentLogWriterStopTime = currentTime + (rollingIntervalSecs * 1000) + val newLogPath = new Path(logDirectory, + timeToLogFile(currentLogWriterStartTime, currentLogWriterStopTime)) + currentLogPath = Some(newLogPath.toString) + currentLogWriter = new WriteAheadLogWriter(currentLogPath.get, hadoopConf) + } + currentLogWriter + } + + /** Initialize the log directory or recover existing logs inside the directory */ + private def initializeOrRecover(): Unit = synchronized { + val logDirectoryPath = new Path(logDirectory) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + + if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { + val logFileInfo = logFilesTologInfo(fileSystem.listStatus(logDirectoryPath).map { _.getPath }) + pastLogs.clear() + pastLogs ++= logFileInfo + logInfo(s"Recovered ${logFileInfo.size} write ahead log files from $logDirectory") + logDebug(s"Recovered files are:\n${logFileInfo.map(_.path).mkString("\n")}") + } + } + + private def resetWriter(): Unit = synchronized { + if (currentLogWriter != null) { + currentLogWriter.close() + currentLogWriter = null + } + } +} + +private[util] object WriteAheadLogManager { + + case class LogInfo(startTime: Long, endTime: Long, path: String) + + val logFileRegex = """log-(\d+)-(\d+)""".r + + def timeToLogFile(startTime: Long, stopTime: Long): String = { + s"log-$startTime-$stopTime" + } + + /** Convert a sequence of files to a sequence of sorted LogInfo objects */ + def logFilesTologInfo(files: Seq[Path]): Seq[LogInfo] = { + files.flatMap { file => + logFileRegex.findFirstIn(file.getName()) match { + case Some(logFileRegex(startTimeStr, stopTimeStr)) => + val startTime = startTimeStr.toLong + val stopTime = stopTimeStr.toLong + Some(LogInfo(startTime, stopTime, file.toString)) + case None => + None + } + }.sortBy { _.startTime } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala new file mode 100644 index 0000000000000..92bad7a882a65 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import java.io.Closeable +import java.nio.ByteBuffer + +import org.apache.hadoop.conf.Configuration + +/** + * A random access reader for reading write ahead log files written using + * [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. Given the file segment info, + * this reads the record (bytebuffer) from the log file. + */ +private[streaming] class WriteAheadLogRandomReader(path: String, conf: Configuration) + extends Closeable { + + private val instream = HdfsUtils.getInputStream(path, conf) + private var closed = false + + def read(segment: WriteAheadLogFileSegment): ByteBuffer = synchronized { + assertOpen() + instream.seek(segment.offset) + val nextLength = instream.readInt() + HdfsUtils.checkState(nextLength == segment.length, + s"Expected message length to be ${segment.length}, but was $nextLength") + val buffer = new Array[Byte](nextLength) + instream.readFully(buffer) + ByteBuffer.wrap(buffer) + } + + override def close(): Unit = synchronized { + closed = true + instream.close() + } + + private def assertOpen() { + HdfsUtils.checkState(!closed, "Stream is closed. Create a new Reader to read from the file.") + } +} + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala new file mode 100644 index 0000000000000..2afc0d1551acf --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.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.streaming.util + +import java.io.{Closeable, EOFException} +import java.nio.ByteBuffer + +import org.apache.hadoop.conf.Configuration +import org.apache.spark.Logging + +/** + * A reader for reading write ahead log files written using + * [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. This reads + * the records (bytebuffers) in the log file sequentially and return them as an + * iterator of bytebuffers. + */ +private[streaming] class WriteAheadLogReader(path: String, conf: Configuration) + extends Iterator[ByteBuffer] with Closeable with Logging { + + private val instream = HdfsUtils.getInputStream(path, conf) + private var closed = false + private var nextItem: Option[ByteBuffer] = None + + override def hasNext: Boolean = synchronized { + if (closed) { + return false + } + + if (nextItem.isDefined) { // handle the case where hasNext is called without calling next + true + } else { + try { + val length = instream.readInt() + val buffer = new Array[Byte](length) + instream.readFully(buffer) + nextItem = Some(ByteBuffer.wrap(buffer)) + logTrace("Read next item " + nextItem.get) + true + } catch { + case e: EOFException => + logDebug("Error reading next item, EOF reached", e) + close() + false + case e: Exception => + logWarning("Error while trying to read data from HDFS.", e) + close() + throw e + } + } + } + + override def next(): ByteBuffer = synchronized { + val data = nextItem.getOrElse { + close() + throw new IllegalStateException( + "next called without calling hasNext or after hasNext returned false") + } + nextItem = None // Ensure the next hasNext call loads new data. + data + } + + override def close(): Unit = synchronized { + if (!closed) { + instream.close() + } + closed = true + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala new file mode 100644 index 0000000000000..679f6a6dfd7c1 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.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.streaming.util + +import java.io._ +import java.net.URI +import java.nio.ByteBuffer + +import scala.util.Try + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FSDataOutputStream, FileSystem} + +/** + * A writer for writing byte-buffers to a write ahead log file. + */ +private[streaming] class WriteAheadLogWriter(path: String, hadoopConf: Configuration) + extends Closeable { + + private lazy val stream = HdfsUtils.getOutputStream(path, hadoopConf) + + private lazy val hadoopFlushMethod = { + // Use reflection to get the right flush operation + val cls = classOf[FSDataOutputStream] + Try(cls.getMethod("hflush")).orElse(Try(cls.getMethod("sync"))).toOption + } + + private var nextOffset = stream.getPos() + private var closed = false + + /** Write the bytebuffer to the log file */ + def write(data: ByteBuffer): WriteAheadLogFileSegment = synchronized { + assertOpen() + data.rewind() // Rewind to ensure all data in the buffer is retrieved + val lengthToWrite = data.remaining() + val segment = new WriteAheadLogFileSegment(path, nextOffset, lengthToWrite) + stream.writeInt(lengthToWrite) + if (data.hasArray) { + stream.write(data.array()) + } else { + // If the buffer is not backed by an array, we transfer using temp array + // Note that despite the extra array copy, this should be faster than byte-by-byte copy + while (data.hasRemaining) { + val array = new Array[Byte](data.remaining) + data.get(array) + stream.write(array) + } + } + flush() + nextOffset = stream.getPos() + segment + } + + override def close(): Unit = synchronized { + closed = true + stream.close() + } + + private def flush() { + hadoopFlushMethod.foreach { _.invoke(stream) } + // Useful for local file system where hflush/sync does not work (HADOOP-7844) + stream.getWrappedStream.flush() + } + + private def assertOpen() { + HdfsUtils.checkState(!closed, "Stream is closed. Create a new Writer to write to file.") + } +} 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 new file mode 100644 index 0000000000000..5eba93c208c50 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -0,0 +1,357 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import java.io._ +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.{implicitConversions, postfixOps} +import scala.util.Random + +import WriteAheadLogSuite._ +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.util.Utils +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually._ + +class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { + + val hadoopConf = new Configuration() + var tempDir: File = null + var testDir: String = null + var testFile: String = null + var manager: WriteAheadLogManager = null + + before { + tempDir = Files.createTempDir() + testDir = tempDir.toString + testFile = new File(tempDir, Random.nextString(10)).toString + if (manager != null) { + manager.stop() + manager = null + } + } + + after { + FileUtils.deleteQuietly(tempDir) + } + + test("WriteAheadLogWriter - writing data") { + val dataToWrite = generateRandomData() + val segments = writeDataUsingWriter(testFile, dataToWrite) + val writtenData = readDataManually(testFile, segments) + assert(writtenData === dataToWrite) + } + + test("WriteAheadLogWriter - syncing of data by writing and reading immediately") { + val dataToWrite = generateRandomData() + val writer = new WriteAheadLogWriter(testFile, hadoopConf) + dataToWrite.foreach { data => + val segment = writer.write(stringToByteBuffer(data)) + val dataRead = readDataManually(testFile, Seq(segment)).head + assert(data === dataRead) + } + writer.close() + } + + test("WriteAheadLogReader - sequentially reading data") { + val writtenData = generateRandomData() + writeDataManually(writtenData, testFile) + val reader = new WriteAheadLogReader(testFile, hadoopConf) + val readData = reader.toSeq.map(byteBufferToString) + assert(readData === writtenData) + assert(reader.hasNext === false) + intercept[Exception] { + reader.next() + } + reader.close() + } + + test("WriteAheadLogReader - sequentially reading data written with writer") { + val dataToWrite = generateRandomData() + writeDataUsingWriter(testFile, dataToWrite) + val readData = readDataUsingReader(testFile) + assert(readData === dataToWrite) + } + + test("WriteAheadLogReader - reading data written with writer after corrupted write") { + // Write data manually for testing the sequential reader + val dataToWrite = generateRandomData() + writeDataUsingWriter(testFile, dataToWrite) + val fileLength = new File(testFile).length() + + // Append some garbage data to get the effect of a corrupted write + val fw = new FileWriter(testFile, true) + fw.append("This line appended to file!") + fw.close() + + // Verify the data can be read and is same as the one correctly written + assert(readDataUsingReader(testFile) === dataToWrite) + + // Corrupt the last correctly written file + val raf = new FileOutputStream(testFile, true).getChannel() + raf.truncate(fileLength - 1) + raf.close() + + // Verify all the data except the last can be read + assert(readDataUsingReader(testFile) === (dataToWrite.dropRight(1))) + } + + test("WriteAheadLogRandomReader - reading data using random reader") { + // Write data manually for testing the random reader + val writtenData = generateRandomData() + val segments = writeDataManually(writtenData, testFile) + + // Get a random order of these segments and read them back + val writtenDataAndSegments = writtenData.zip(segments).toSeq.permutations.take(10).flatten + val reader = new WriteAheadLogRandomReader(testFile, hadoopConf) + writtenDataAndSegments.foreach { case (data, segment) => + assert(data === byteBufferToString(reader.read(segment))) + } + reader.close() + } + + test("WriteAheadLogRandomReader - reading data using random reader written with writer") { + // Write data using writer for testing the random reader + val data = generateRandomData() + val segments = writeDataUsingWriter(testFile, data) + + // Read a random sequence of segments and verify read data + val dataAndSegments = data.zip(segments).toSeq.permutations.take(10).flatten + val reader = new WriteAheadLogRandomReader(testFile, hadoopConf) + dataAndSegments.foreach { case (data, segment) => + assert(data === byteBufferToString(reader.read(segment))) + } + reader.close() + } + + test("WriteAheadLogManager - write rotating logs") { + // Write data using manager + val dataToWrite = generateRandomData() + writeDataUsingManager(testDir, dataToWrite) + + // Read data manually to verify the written data + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + val writtenData = logFiles.flatMap { file => readDataManually(file)} + assert(writtenData === dataToWrite) + } + + test("WriteAheadLogManager - read rotating logs") { + // Write data manually for testing reading through manager + val writtenData = (1 to 10).map { i => + val data = generateRandomData() + val file = testDir + s"/log-$i-$i" + writeDataManually(data, file) + data + }.flatten + + val logDirectoryPath = new Path(testDir) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + assert(fileSystem.exists(logDirectoryPath) === true) + + // Read data using manager and verify + val readData = readDataUsingManager(testDir) + assert(readData === writtenData) + } + + test("WriteAheadLogManager - recover past logs when creating new manager") { + // Write data with manager, recover with new manager and verify + val dataToWrite = generateRandomData() + writeDataUsingManager(testDir, dataToWrite) + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + val readData = readDataUsingManager(testDir) + assert(dataToWrite === readData) + } + + test("WriteAheadLogManager - cleanup old logs") { + // Write data with manager, recover with new manager and verify + val manualClock = new ManualClock + val dataToWrite = generateRandomData() + manager = writeDataUsingManager(testDir, dataToWrite, manualClock, stopManager = false) + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + manager.cleanupOldLogs(manualClock.currentTime() / 2) + eventually(timeout(1 second), interval(10 milliseconds)) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } + } + + test("WriteAheadLogManager - handling file errors while reading rotating logs") { + // Generate a set of log files + val manualClock = new ManualClock + val dataToWrite1 = generateRandomData() + writeDataUsingManager(testDir, dataToWrite1, manualClock) + val logFiles1 = getLogFilesInDirectory(testDir) + assert(logFiles1.size > 1) + + + // Recover old files and generate a second set of log files + val dataToWrite2 = generateRandomData() + manualClock.addToTime(100000) + writeDataUsingManager(testDir, dataToWrite2, manualClock) + val logFiles2 = getLogFilesInDirectory(testDir) + assert(logFiles2.size > logFiles1.size) + + // Read the files and verify that all the written data can be read + val readData1 = readDataUsingManager(testDir) + assert(readData1 === (dataToWrite1 ++ dataToWrite2)) + + // Corrupt the first set of files so that they are basically unreadable + logFiles1.foreach { f => + val raf = new FileOutputStream(f, true).getChannel() + raf.truncate(1) + raf.close() + } + + // Verify that the corrupted files do not prevent reading of the second set of data + val readData = readDataUsingManager(testDir) + assert(readData === dataToWrite2) + } +} + +object WriteAheadLogSuite { + + private val hadoopConf = new Configuration() + + /** Write data to a file directly and return an array of the file segments written. */ + def writeDataManually(data: Seq[String], file: String): Seq[WriteAheadLogFileSegment] = { + val segments = new ArrayBuffer[WriteAheadLogFileSegment]() + val writer = HdfsUtils.getOutputStream(file, hadoopConf) + data.foreach { item => + val offset = writer.getPos + val bytes = Utils.serialize(item) + writer.writeInt(bytes.size) + writer.write(bytes) + segments += WriteAheadLogFileSegment(file, offset, bytes.size) + } + writer.close() + segments + } + + /** + * Write data to a file using the writer class and return an array of the file segments written. + */ + def writeDataUsingWriter(filePath: String, data: Seq[String]): Seq[WriteAheadLogFileSegment] = { + val writer = new WriteAheadLogWriter(filePath, hadoopConf) + val segments = data.map { + item => writer.write(item) + } + writer.close() + segments + } + + /** Write data to rotating files in log directory using the manager class. */ + def writeDataUsingManager( + logDirectory: String, + data: Seq[String], + manualClock: ManualClock = new ManualClock, + stopManager: Boolean = true + ): WriteAheadLogManager = { + if (manualClock.currentTime < 100000) manualClock.setTime(10000) + val manager = new WriteAheadLogManager(logDirectory, hadoopConf, + rollingIntervalSecs = 1, callerName = "WriteAheadLogSuite", clock = manualClock) + // Ensure that 500 does not get sorted after 2000, so put a high base value. + data.foreach { item => + manualClock.addToTime(500) + manager.writeToLog(item) + } + if (stopManager) manager.stop() + manager + } + + /** Read data from a segments of a log file directly and return the list of byte buffers.*/ + def readDataManually(file: String, segments: Seq[WriteAheadLogFileSegment]): Seq[String] = { + val reader = HdfsUtils.getInputStream(file, hadoopConf) + segments.map { x => + reader.seek(x.offset) + val data = new Array[Byte](x.length) + reader.readInt() + reader.readFully(data) + Utils.deserialize[String](data) + } + } + + /** Read all the data from a log file directly and return the list of byte buffers. */ + def readDataManually(file: String): Seq[String] = { + val reader = HdfsUtils.getInputStream(file, hadoopConf) + val buffer = new ArrayBuffer[String] + try { + while (true) { + // Read till EOF is thrown + val length = reader.readInt() + val bytes = new Array[Byte](length) + reader.read(bytes) + buffer += Utils.deserialize[String](bytes) + } + } catch { + case ex: EOFException => + } finally { + reader.close() + } + buffer + } + + /** Read all the data from a log file using reader class and return the list of byte buffers. */ + def readDataUsingReader(file: String): Seq[String] = { + val reader = new WriteAheadLogReader(file, hadoopConf) + val readData = reader.toList.map(byteBufferToString) + reader.close() + readData + } + + /** Read all the data in the log file in a directory using the manager class. */ + def readDataUsingManager(logDirectory: String): Seq[String] = { + val manager = new WriteAheadLogManager(logDirectory, hadoopConf, + callerName = "WriteAheadLogSuite") + val data = manager.readFromLog().map(byteBufferToString).toSeq + manager.stop() + data + } + + /** Get the log files in a direction */ + def getLogFilesInDirectory(directory: String): Seq[String] = { + val logDirectoryPath = new Path(directory) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + + if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { + fileSystem.listStatus(logDirectoryPath).map { + _.getPath.toString.stripPrefix("file:") + }.sorted + } else { + Seq.empty + } + } + + def generateRandomData(): Seq[String] = { + (1 to 100).map { _.toString } + } + + implicit def stringToByteBuffer(str: String): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize(str)) + } + + implicit def byteBufferToString(byteBuffer: ByteBuffer): String = { + Utils.deserialize[String](byteBuffer.array) + } +} From 7aacb7bfad4ec73fd8f18555c72ef6962c14358f Mon Sep 17 00:00:00 2001 From: Li Zhihui Date: Fri, 24 Oct 2014 13:01:36 -0700 Subject: [PATCH 010/115] [SPARK-2713] Executors of same application in same host should only download files & jars once If Spark lunched multiple executors in one host for one application, every executor would download it dependent files and jars (if not using local: url) independently. It maybe result in huge latency. In my case, it result in 20 seconds latency to download dependent jars(size about 17M) when I lunched 32 executors in every host(total 4 hosts). This patch will cache downloaded files and jars for executors to reduce network throughput and download latency. In my case, the latency was reduced from 20 seconds to less than 1 second. Author: Li Zhihui Author: li-zhihui Closes #1616 from li-zhihui/cachefiles and squashes the following commits: 36940df [Li Zhihui] Close cache for local mode 935fed6 [Li Zhihui] Clean code. f9330d4 [Li Zhihui] Clean code again 7050d46 [Li Zhihui] Clean code 074a422 [Li Zhihui] Fix: deal with spark.files.overwrite 03ed3a8 [li-zhihui] rename cache file name as XXXXXXXXX_cache 2766055 [li-zhihui] Use url.hashCode + timestamp as cachedFileName 76a7b66 [Li Zhihui] Clean code & use applcation work directory as cache directory 3510eb0 [Li Zhihui] Keep fetchFile private 2ffd742 [Li Zhihui] add comment for FileLock e0ebd48 [Li Zhihui] Try and finally lock.release 7fb7c0b [Li Zhihui] Release lock before copy files 6b997bf [Li Zhihui] Executors of same application in same host should only download files & jars once --- .../scala/org/apache/spark/SparkContext.scala | 5 +- .../org/apache/spark/executor/Executor.scala | 10 ++- .../scala/org/apache/spark/util/Utils.scala | 87 ++++++++++++++++--- 3 files changed, 82 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ac7935b8c231e..55602a90829d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -837,11 +837,12 @@ class SparkContext(config: SparkConf) extends Logging { case "local" => "file:" + uri.getPath case _ => path } - addedFiles(key) = System.currentTimeMillis + val timestamp = System.currentTimeMillis + addedFiles(key) = timestamp // Fetch the file locally in case a job is executed using DAGScheduler.runLocally(). Utils.fetchFile(path, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, - hadoopConfiguration) + hadoopConfiguration, timestamp, useCache = false) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) postEnvironmentUpdate() 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 616c7e6a46368..0b75b9b21fb82 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -322,14 +322,16 @@ private[spark] class Executor( // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, env.securityManager, - hadoopConf) + // Fetch file with useCache mode, close cache for local mode. + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentFiles(name) = timestamp } for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, env.securityManager, - hadoopConf) + // Fetch file with useCache mode, close cache for local mode. + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentJars(name) = timestamp // Add it to our class loader val localName = name.split("/").last 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 0aeff6455b3fe..ccbddd985ae0a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -347,15 +347,84 @@ private[spark] object Utils extends Logging { } /** - * Download a file requested by the executor. Supports fetching the file in a variety of ways, + * Download a file to target directory. Supports fetching the file in a variety of ways, + * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. + * + * If `useCache` is true, first attempts to fetch the file to a local cache that's shared + * across executors running the same application. `useCache` is used mainly for + * the executors, and not in local mode. + * + * Throws SparkException if the target file already exists and has different contents than + * the requested file. + */ + def fetchFile( + url: String, + targetDir: File, + conf: SparkConf, + securityMgr: SecurityManager, + hadoopConf: Configuration, + timestamp: Long, + useCache: Boolean) { + val fileName = url.split("/").last + val targetFile = new File(targetDir, fileName) + if (useCache) { + val cachedFileName = s"${url.hashCode}${timestamp}_cache" + val lockFileName = s"${url.hashCode}${timestamp}_lock" + val localDir = new File(getLocalDir(conf)) + val lockFile = new File(localDir, lockFileName) + val raf = new RandomAccessFile(lockFile, "rw") + // Only one executor entry. + // The FileLock is only used to control synchronization for executors download file, + // it's always safe regardless of lock type (mandatory or advisory). + val lock = raf.getChannel().lock() + val cachedFile = new File(localDir, cachedFileName) + try { + if (!cachedFile.exists()) { + doFetchFile(url, localDir, cachedFileName, conf, securityMgr, hadoopConf) + } + } finally { + lock.release() + } + if (targetFile.exists && !Files.equal(cachedFile, targetFile)) { + if (conf.getBoolean("spark.files.overwrite", false)) { + targetFile.delete() + logInfo((s"File $targetFile exists and does not match contents of $url, " + + s"replacing it with $url")) + } else { + throw new SparkException(s"File $targetFile exists and does not match contents of $url") + } + } + Files.copy(cachedFile, targetFile) + } else { + doFetchFile(url, targetDir, fileName, conf, securityMgr, hadoopConf) + } + + // Decompress the file if it's a .tar or .tar.gz + if (fileName.endsWith(".tar.gz") || fileName.endsWith(".tgz")) { + logInfo("Untarring " + fileName) + Utils.execute(Seq("tar", "-xzf", fileName), targetDir) + } else if (fileName.endsWith(".tar")) { + logInfo("Untarring " + fileName) + Utils.execute(Seq("tar", "-xf", fileName), targetDir) + } + // Make the file executable - That's necessary for scripts + FileUtil.chmod(targetFile.getAbsolutePath, "a+x") + } + + /** + * Download a file to target directory. Supports fetching the file in a variety of ways, * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. * * Throws SparkException if the target file already exists and has different contents than * the requested file. */ - def fetchFile(url: String, targetDir: File, conf: SparkConf, securityMgr: SecurityManager, - hadoopConf: Configuration) { - val filename = url.split("/").last + private def doFetchFile( + url: String, + targetDir: File, + filename: String, + conf: SparkConf, + securityMgr: SecurityManager, + hadoopConf: Configuration) { val tempDir = getLocalDir(conf) val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) val targetFile = new File(targetDir, filename) @@ -443,16 +512,6 @@ private[spark] object Utils extends Logging { } Files.move(tempFile, targetFile) } - // Decompress the file if it's a .tar or .tar.gz - if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) { - logInfo("Untarring " + filename) - Utils.execute(Seq("tar", "-xzf", filename), targetDir) - } else if (filename.endsWith(".tar")) { - logInfo("Untarring " + filename) - Utils.execute(Seq("tar", "-xf", filename), targetDir) - } - // Make the file executable - That's necessary for scripts - FileUtil.chmod(targetFile.getAbsolutePath, "a+x") } /** From 30ea2868e7afbec20bfc83818249b6d2d7dc6aec Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 24 Oct 2014 13:04:35 -0700 Subject: [PATCH 011/115] [SPARK-4076] Parameter expansion in spark-config is wrong In sbin/spark-config.sh, parameter expansion is used to extract source root as follows. this="${BASH_SOURCE-$0}" I think, the parameter expansion should be ":" instead of "". If we use "-" and BASH_SOURCE="", (empty character is set, not unset), "" (empty character) is set to $this. Author: Kousuke Saruta Closes #2930 from sarutak/SPARK-4076 and squashes the following commits: 32a0370 [Kousuke Saruta] Fixed wrong parameter expansion --- sbin/spark-config.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 1d154e62ed5b6..b0361d72d3f2c 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -20,7 +20,7 @@ # also should not be passed any arguments, since we need original $* # resolve links - $0 may be a softlink -this="${BASH_SOURCE-$0}" +this="${BASH_SOURCE:-$0}" common_bin="$(cd -P -- "$(dirname -- "$this")" && pwd -P)" script="$(basename -- "$this")" this="$common_bin/$script" From 098f83c7ccd7dad9f9228596da69fe5f55711a52 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 24 Oct 2014 13:08:21 -0700 Subject: [PATCH 012/115] [SPARK-4075] [Deploy] Jar url validation is not enough for Jar file In deploy.ClientArguments.isValidJarUrl, the url is checked as follows. def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar") So, it allows like 'hdfs:file.jar' (no authority). Author: Kousuke Saruta Closes #2925 from sarutak/uri-syntax-check-improvement and squashes the following commits: cf06173 [Kousuke Saruta] Improved URI syntax checking --- .../org/apache/spark/deploy/ClientArguments.scala | 11 ++++++++++- .../scala/org/apache/spark/deploy/ClientSuite.scala | 6 ++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 39150deab863c..4e802e02c4149 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy +import java.net.{URI, URISyntaxException} + import scala.collection.mutable.ListBuffer import org.apache.log4j.Level @@ -114,5 +116,12 @@ private[spark] class ClientArguments(args: Array[String]) { } object ClientArguments { - def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar") + def isValidJarUrl(s: String): Boolean = { + try { + val uri = new URI(s) + uri.getScheme != null && uri.getAuthority != null && s.endsWith("jar") + } catch { + case _: URISyntaxException => false + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 4161aede1d1d0..94a2bdd74e744 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -29,6 +29,12 @@ class ClientSuite extends FunSuite with Matchers { ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo") should be (false) ClientArguments.isValidJarUrl("/missing/a/protocol/jarfile.jar") should be (false) ClientArguments.isValidJarUrl("not-even-a-path.jar") should be (false) + + // No authority + ClientArguments.isValidJarUrl("hdfs:someHost:1234/jarfile.jar") should be (false) + + // Invalid syntax + ClientArguments.isValidJarUrl("hdfs:") should be (false) } } From b563987e8dffc2aed1a834d555589a41cfb2a706 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 24 Oct 2014 13:32:23 -0700 Subject: [PATCH 013/115] [SPARK-4013] Do not create multiple actor systems on each executor In the existing code, each coarse-grained executor has two concurrently running actor systems. This causes many more error messages to be logged than necessary when the executor is lost or killed because we receive a disassociation event for each of these actor systems. This is blocking #2840. Author: Andrew Or Closes #2863 from andrewor14/executor-actor-system and squashes the following commits: 44ce2e0 [Andrew Or] Avoid starting two actor systems on each executor --- .../scala/org/apache/spark/SparkContext.scala | 12 ++--- .../scala/org/apache/spark/SparkEnv.scala | 49 +++++++++++++++++-- .../CoarseGrainedExecutorBackend.scala | 11 +++-- .../org/apache/spark/executor/Executor.scala | 11 +++-- 4 files changed, 61 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 55602a90829d5..4565832334420 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -209,16 +209,10 @@ class SparkContext(config: SparkConf) extends Logging { // An asynchronous listener bus for Spark events private[spark] val listenerBus = new LiveListenerBus - // Create the Spark execution environment (cache, map output tracker, etc) conf.set("spark.executor.id", "driver") - private[spark] val env = SparkEnv.create( - conf, - "", - conf.get("spark.driver.host"), - conf.get("spark.driver.port").toInt, - isDriver = true, - isLocal = isLocal, - listenerBus = listenerBus) + + // Create the Spark execution environment (cache, map output tracker, etc) + private[spark] val env = SparkEnv.createDriverEnv(conf, isLocal, listenerBus) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 906a00b0bd17c..5c076e5f1c11d 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -144,14 +144,46 @@ object SparkEnv extends Logging { env } - private[spark] def create( + /** + * Create a SparkEnv for the driver. + */ + private[spark] def createDriverEnv( + conf: SparkConf, + isLocal: Boolean, + listenerBus: LiveListenerBus): SparkEnv = { + assert(conf.contains("spark.driver.host"), "spark.driver.host is not set on the driver!") + assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") + val hostname = conf.get("spark.driver.host") + val port = conf.get("spark.driver.port").toInt + create(conf, "", hostname, port, true, isLocal, listenerBus) + } + + /** + * Create a SparkEnv for an executor. + * In coarse-grained mode, the executor provides an actor system that is already instantiated. + */ + private[spark] def createExecutorEnv( + conf: SparkConf, + executorId: String, + hostname: String, + port: Int, + isLocal: Boolean, + actorSystem: ActorSystem = null): SparkEnv = { + create(conf, executorId, hostname, port, false, isLocal, defaultActorSystem = actorSystem) + } + + /** + * Helper method to create a SparkEnv for a driver or an executor. + */ + private def create( conf: SparkConf, executorId: String, hostname: String, port: Int, isDriver: Boolean, isLocal: Boolean, - listenerBus: LiveListenerBus = null): SparkEnv = { + listenerBus: LiveListenerBus = null, + defaultActorSystem: ActorSystem = null): SparkEnv = { // Listener bus is only used on the driver if (isDriver) { @@ -159,9 +191,16 @@ object SparkEnv extends Logging { } val securityManager = new SecurityManager(conf) - val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - actorSystemName, hostname, port, conf, securityManager) + + // If an existing actor system is already provided, use it. + // This is the case when an executor is launched in coarse-grained mode. + val (actorSystem, boundPort) = + Option(defaultActorSystem) match { + case Some(as) => (as, port) + case None => + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) + } // Figure out which port Akka actually bound to in case the original port is 0 or occupied. // This is so that we tell the executors the correct port to connect to. 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 c40a3e16675ad..697154d762d41 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import scala.concurrent.Await -import akka.actor.{Actor, ActorSelection, Props} +import akka.actor.{Actor, ActorSelection, ActorSystem, Props} import akka.pattern.Patterns import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} @@ -38,7 +38,8 @@ private[spark] class CoarseGrainedExecutorBackend( executorId: String, hostPort: String, cores: Int, - sparkProperties: Seq[(String, String)]) + sparkProperties: Seq[(String, String)], + actorSystem: ActorSystem) extends Actor with ActorLogReceive with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") @@ -57,8 +58,8 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, - false) + val (hostname, _) = Utils.parseHostPort(hostPort) + executor = new Executor(executorId, hostname, sparkProperties, isLocal = false, actorSystem) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -135,7 +136,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], - driverUrl, executorId, sparkHostPort, cores, props), + driverUrl, executorId, sparkHostPort, cores, props, actorSystem), name = "Executor") workerUrl.foreach { url => actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") 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 0b75b9b21fb82..70a46c75f42c4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -26,6 +26,8 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal +import akka.actor.ActorSystem + import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ @@ -35,12 +37,14 @@ import org.apache.spark.util.{AkkaUtils, Utils} /** * Spark executor used with Mesos, YARN, and the standalone scheduler. + * In coarse-grained mode, an existing actor system is provided. */ private[spark] class Executor( executorId: String, slaveHostname: String, properties: Seq[(String, String)], - isLocal: Boolean = false) + isLocal: Boolean = false, + actorSystem: ActorSystem = null) extends Logging { // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -77,8 +81,9 @@ private[spark] class Executor( conf.set("spark.executor.id", "executor." + executorId) private val env = { if (!isLocal) { - val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, - isDriver = false, isLocal = false) + val port = conf.getInt("spark.executor.port", 0) + val _env = SparkEnv.createExecutorEnv( + conf, executorId, slaveHostname, port, isLocal, actorSystem) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) _env From f80dcf2aeef762ca370e91d2c7d6e4f7894c3cd8 Mon Sep 17 00:00:00 2001 From: Nan Zhu Date: Fri, 24 Oct 2014 13:46:45 -0700 Subject: [PATCH 014/115] [SPARK-4067] refactor ExecutorUncaughtExceptionHandler https://issues.apache.org/jira/browse/SPARK-4067 currently , we call Utils.tryOrExit everywhere AppClient Executor TaskSchedulerImpl It makes the name of ExecutorUncaughtExceptionHandler unfit to the real case.... Author: Nan Zhu Author: Nan Zhu Closes #2913 from CodingCat/SPARK-4067 and squashes the following commits: 035ee3d [Nan Zhu] make RAT happy e62e416 [Nan Zhu] add some general Exit code a10b63f [Nan Zhu] refactor --- .../org/apache/spark/executor/Executor.scala | 6 ++-- .../spark/executor/ExecutorExitCode.scala | 12 ++----- .../org/apache/spark/util/SparkExitCode.scala | 32 +++++++++++++++++++ .../SparkUncaughtExceptionHandler.scala} | 13 ++++---- .../scala/org/apache/spark/util/Utils.scala | 4 +-- 5 files changed, 45 insertions(+), 22 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/SparkExitCode.scala rename core/src/main/scala/org/apache/spark/{executor/ExecutorUncaughtExceptionHandler.scala => util/SparkUncaughtExceptionHandler.scala} (80%) 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 70a46c75f42c4..2889f59e33e84 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -33,7 +33,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{SparkUncaughtExceptionHandler, AkkaUtils, Utils} /** * Spark executor used with Mesos, YARN, and the standalone scheduler. @@ -72,7 +72,7 @@ private[spark] class Executor( // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. - Thread.setDefaultUncaughtExceptionHandler(ExecutorUncaughtExceptionHandler) + Thread.setDefaultUncaughtExceptionHandler(SparkUncaughtExceptionHandler) } val executorSource = new ExecutorSource(this, executorId) @@ -258,7 +258,7 @@ private[spark] class Executor( // Don't forcibly exit unless the exception was inherently fatal, to avoid // stopping other tasks unnecessarily. if (Utils.isFatalError(t)) { - ExecutorUncaughtExceptionHandler.uncaughtException(t) + SparkUncaughtExceptionHandler.uncaughtException(t) } } } finally { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index 38be2c58b333f..52862ae0ca5e4 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -17,6 +17,8 @@ package org.apache.spark.executor +import org.apache.spark.util.SparkExitCode._ + /** * These are exit codes that executors should use to provide the master with information about * executor failures assuming that cluster management framework can capture the exit codes (but @@ -27,16 +29,6 @@ package org.apache.spark.executor */ private[spark] object ExecutorExitCode { - /** The default uncaught exception handler was reached. */ - val UNCAUGHT_EXCEPTION = 50 - - /** The default uncaught exception handler was called and an exception was encountered while - logging the exception. */ - val UNCAUGHT_EXCEPTION_TWICE = 51 - - /** The default uncaught exception handler was reached, and the uncaught exception was an - OutOfMemoryError. */ - val OOM = 52 /** DiskStore failed to create a local temporary directory after many attempts. */ val DISK_STORE_FAILED_TO_CREATE_DIR = 53 diff --git a/core/src/main/scala/org/apache/spark/util/SparkExitCode.scala b/core/src/main/scala/org/apache/spark/util/SparkExitCode.scala new file mode 100644 index 0000000000000..c93b1cca9f564 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SparkExitCode.scala @@ -0,0 +1,32 @@ +/* + * 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.util + +private[spark] object SparkExitCode { + /** The default uncaught exception handler was reached. */ + val UNCAUGHT_EXCEPTION = 50 + + /** The default uncaught exception handler was called and an exception was encountered while + logging the exception. */ + val UNCAUGHT_EXCEPTION_TWICE = 51 + + /** The default uncaught exception handler was reached, and the uncaught exception was an + OutOfMemoryError. */ + val OOM = 52 + +} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala similarity index 80% rename from core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala rename to core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala index b0e984c03964c..ad3db1fbb57ed 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala @@ -15,17 +15,16 @@ * limitations under the License. */ -package org.apache.spark.executor +package org.apache.spark.util import org.apache.spark.Logging -import org.apache.spark.util.Utils /** * The default uncaught exception handler for Executors terminates the whole process, to avoid * getting into a bad state indefinitely. Since Executors are relatively lightweight, it's better * to fail fast when things go wrong. */ -private[spark] object ExecutorUncaughtExceptionHandler +private[spark] object SparkUncaughtExceptionHandler extends Thread.UncaughtExceptionHandler with Logging { override def uncaughtException(thread: Thread, exception: Throwable) { @@ -36,14 +35,14 @@ private[spark] object ExecutorUncaughtExceptionHandler // (If we do, we will deadlock.) if (!Utils.inShutdown()) { if (exception.isInstanceOf[OutOfMemoryError]) { - System.exit(ExecutorExitCode.OOM) + System.exit(SparkExitCode.OOM) } else { - System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + System.exit(SparkExitCode.UNCAUGHT_EXCEPTION) } } } catch { - case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) - case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) + case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM) + case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE) } } 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 ccbddd985ae0a..65bdbaae65463 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -43,7 +43,7 @@ import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark._ -import org.apache.spark.executor.ExecutorUncaughtExceptionHandler +import org.apache.spark.util.SparkUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ @@ -965,7 +965,7 @@ private[spark] object Utils extends Logging { block } catch { case e: ControlThrowable => throw e - case t: Throwable => ExecutorUncaughtExceptionHandler.uncaughtException(t) + case t: Throwable => SparkUncaughtExceptionHandler.uncaughtException(t) } } From 07e439b4fe1cea4cee8ec8e39803b8349078f119 Mon Sep 17 00:00:00 2001 From: Grace Date: Fri, 24 Oct 2014 13:48:08 -0700 Subject: [PATCH 015/115] [GraphX] Modify option name according to example doc in SynthBenchmark Now graphx.SynthBenchmark example has an option of iteration number named as "niter". However, in its document, it is named as "niters". The mismatch between the implementation and document causes certain IllegalArgumentException while trying that example. Author: Grace Closes #2888 from GraceH/synthbenchmark and squashes the following commits: f101ee1 [Grace] Modify option name according to example doc --- .../scala/org/apache/spark/examples/graphx/SynthBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 05676021718d9..3ec20d594b784 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -67,7 +67,7 @@ object SynthBenchmark { options.foreach { case ("app", v) => app = v - case ("niter", v) => niter = v.toInt + case ("niters", v) => niter = v.toInt case ("nverts", v) => numVertices = v.toInt case ("numEPart", v) => numEPart = Some(v.toInt) case ("partStrategy", v) => partitionStrategy = Some(PartitionStrategy.fromString(v)) From 3a906c6631a914da8ede3111c63f89a0dac3f369 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 24 Oct 2014 14:03:03 -0700 Subject: [PATCH 016/115] [HOTFIX][SQL] Remove sleep on reset() failure. Author: Michael Armbrust Closes #2934 from marmbrus/patch-2 and squashes the following commits: a96dab2 [Michael Armbrust] Remove sleep on reset() failure. --- .../src/main/scala/org/apache/spark/sql/hive/TestHive.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 0f74fe8943706..c6ff4ea6de594 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -438,10 +438,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { } catch { case e: Exception => logError("FATAL ERROR: Failed to reset TestDB state.", e) - // At this point there is really no reason to continue, but the test framework traps exits. - // So instead we just pause forever so that at least the developer can see where things - // started to go wrong. - Thread.sleep(100000) } } } From 6c98c29ae0033556fd4424f41d1de005c509e511 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 Oct 2014 15:06:15 -0700 Subject: [PATCH 017/115] [SPARK-4080] Only throw IOException from [write|read][Object|External] If classes implementing Serializable or Externalizable interfaces throw exceptions other than IOException or ClassNotFoundException from their (de)serialization methods, then this results in an unhelpful "IOException: unexpected exception type" rather than the actual exception that produced the (de)serialization error. This patch fixes this by adding a utility method that re-wraps any uncaught exceptions in IOException (unless they are already instances of IOException). Author: Josh Rosen Closes #2932 from JoshRosen/SPARK-4080 and squashes the following commits: cd3a9be [Josh Rosen] [SPARK-4080] Only throw IOException from [write|read][Object|External]. --- .../scala/org/apache/spark/Accumulators.scala | 3 ++- .../main/scala/org/apache/spark/Partitioner.scala | 4 ++-- .../org/apache/spark/SerializableWritable.scala | 5 +++-- .../apache/spark/broadcast/HttpBroadcast.scala | 4 ++-- .../apache/spark/broadcast/TorrentBroadcast.scala | 6 +++--- .../spark/deploy/master/ApplicationInfo.scala | 3 ++- .../apache/spark/deploy/master/DriverInfo.scala | 3 ++- .../apache/spark/deploy/master/WorkerInfo.scala | 2 +- .../scala/org/apache/spark/rdd/CartesianRDD.scala | 3 ++- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 3 ++- .../scala/org/apache/spark/rdd/CoalescedRDD.scala | 3 ++- .../apache/spark/rdd/ParallelCollectionRDD.scala | 4 ++-- .../spark/rdd/PartitionerAwareUnionRDD.scala | 3 ++- .../scala/org/apache/spark/rdd/UnionRDD.scala | 3 ++- .../apache/spark/rdd/ZippedPartitionsRDD.scala | 3 ++- .../org/apache/spark/scheduler/MapStatus.scala | 9 +++++---- .../org/apache/spark/scheduler/TaskResult.scala | 4 ++-- .../apache/spark/serializer/JavaSerializer.scala | 4 ++-- .../org/apache/spark/storage/BlockManagerId.scala | 4 ++-- .../spark/storage/BlockManagerMessages.scala | 6 ++++-- .../org/apache/spark/storage/StorageLevel.scala | 5 +++-- .../apache/spark/util/SerializableBuffer.scala | 4 ++-- .../main/scala/org/apache/spark/util/Utils.scala | 15 +++++++++++++++ .../spark/streaming/flume/FlumeInputDStream.scala | 4 ++-- .../org/apache/spark/streaming/DStreamGraph.scala | 5 +++-- .../streaming/api/python/PythonDStream.scala | 5 +++-- .../apache/spark/streaming/dstream/DStream.scala | 6 +++--- .../streaming/dstream/DStreamCheckpointData.scala | 5 +++-- .../streaming/dstream/FileInputDStream.scala | 4 ++-- .../apache/spark/streaming/TestSuiteBase.scala | 4 ++-- 30 files changed, 84 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 12f2fe031cb1d..2301caafb07ff 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -24,6 +24,7 @@ import scala.collection.mutable.Map import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.Utils /** * A data type that can be accumulated, ie has an commutative and associative "add" operation, @@ -126,7 +127,7 @@ class Accumulable[R, T] ( } // Called by Java when deserializing an object - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() value_ = zero deserialized = true diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 37053bb6f37ad..e53a78ead2c0e 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -204,7 +204,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } @throws(classOf[IOException]) - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { val sfactory = SparkEnv.get.serializer sfactory match { case js: JavaSerializer => out.defaultWriteObject() @@ -222,7 +222,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } @throws(classOf[IOException]) - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val sfactory = SparkEnv.get.serializer sfactory match { case js: JavaSerializer => in.defaultReadObject() diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index e50b9ac2291f9..55cb25946c2ad 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -24,18 +24,19 @@ import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils @DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { out.defaultWriteObject() new ObjectWritable(t).write(out) } - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() val ow = new ObjectWritable() ow.setConf(new Configuration()) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 4cd4f4f96fd16..7dade04273b08 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -72,13 +72,13 @@ private[spark] class HttpBroadcast[T: ClassTag]( } /** Used by the JVM when serializing this object. */ - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { assertValid() out.defaultWriteObject() } /** Used by the JVM when deserializing this object. */ - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() HttpBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(blockId) match { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 99af2e9608ea7..75e64c1bf401e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -28,7 +28,7 @@ import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} -import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.{ByteBufferInputStream, Utils} import org.apache.spark.util.io.ByteArrayChunkOutputStream /** @@ -152,13 +152,13 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) } /** Used by the JVM when serializing this object. */ - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { assertValid() out.defaultWriteObject() } /** Used by the JVM when deserializing this object. */ - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() TorrentBroadcast.synchronized { setConf(SparkEnv.get.conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index c3ca43f8d0734..6ba395be1cc2c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import akka.actor.ActorRef import org.apache.spark.deploy.ApplicationDescription +import org.apache.spark.util.Utils private[spark] class ApplicationInfo( val startTime: Long, @@ -46,7 +47,7 @@ private[spark] class ApplicationInfo( init() - private def readObject(in: java.io.ObjectInputStream): Unit = { + private def readObject(in: java.io.ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() init() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index 80b570a44af18..2ac21186881fa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import java.util.Date import org.apache.spark.deploy.DriverDescription +import org.apache.spark.util.Utils private[spark] class DriverInfo( val startTime: Long, @@ -36,7 +37,7 @@ private[spark] class DriverInfo( init() - private def readObject(in: java.io.ObjectInputStream): Unit = { + private def readObject(in: java.io.ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() init() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index c5fa9cf7d7c2d..d221b0f6cc86b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -50,7 +50,7 @@ private[spark] class WorkerInfo( def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed - private def readObject(in: java.io.ObjectInputStream) : Unit = { + private def readObject(in: java.io.ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() init() } diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 4908711d17db7..1cbd684224b7c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -22,6 +22,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.reflect.ClassTag import org.apache.spark._ +import org.apache.spark.util.Utils private[spark] class CartesianPartition( @@ -36,7 +37,7 @@ class CartesianPartition( override val index: Int = idx @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent split at the time of task serialization s1 = rdd1.partitions(s1Index) s2 = rdd2.partitions(s2Index) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index fabb882cdd4b3..ffc0a8a6d67eb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -27,6 +27,7 @@ import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap, CompactBuffer} +import org.apache.spark.util.Utils import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleHandle @@ -39,7 +40,7 @@ private[spark] case class NarrowCoGroupSplitDep( ) extends CoGroupSplitDep { @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent split at the time of task serialization split = rdd.partitions(splitIndex) oos.defaultWriteObject() 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 11ebafbf6d457..9fab1d78abb04 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -25,6 +25,7 @@ import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ +import org.apache.spark.util.Utils /** * Class that captures a coalesced RDD by essentially keeping track of parent partitions @@ -42,7 +43,7 @@ private[spark] case class CoalescedRDDPartition( var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent partition at the time of task serialization parents = parentsIndices.map(rdd.partitions(_)) oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 66c71bf7e8bb5..87b22de6ae697 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -48,7 +48,7 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( override def index: Int = slice @throws(classOf[IOException]) - private def writeObject(out: ObjectOutputStream): Unit = { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { val sfactory = SparkEnv.get.serializer @@ -67,7 +67,7 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( } @throws(classOf[IOException]) - private def readObject(in: ObjectInputStream): Unit = { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val sfactory = SparkEnv.get.serializer sfactory match { diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 0c2cd7a24783b..92b0641d0fb6e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -22,6 +22,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.reflect.ClassTag import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} +import org.apache.spark.util.Utils /** * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of @@ -38,7 +39,7 @@ class PartitionerAwareUnionRDDPartition( override def hashCode(): Int = idx @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent partition at the time of task serialization parents = rdds.map(_.partitions(index)).toArray oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 0c97eb0aaa51f..aece683ff3199 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * Partition for UnionRDD. @@ -48,7 +49,7 @@ private[spark] class UnionPartition[T: ClassTag]( override val index: Int = idx @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent split at the time of task serialization parentPartition = rdd.partitions(parentRddPartitionIndex) oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index f3d30f6c9b32f..996f2cd3f34a3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -22,6 +22,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.reflect.ClassTag import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} +import org.apache.spark.util.Utils private[spark] class ZippedPartitionsPartition( idx: Int, @@ -34,7 +35,7 @@ private[spark] class ZippedPartitionsPartition( def partitions = partitionValues @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent split at the time of task serialization partitionValues = rdds.map(rdd => rdd.partitions(idx)) oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 2ab5d9637b593..01d5943d777f3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -22,6 +22,7 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import org.roaringbitmap.RoaringBitmap import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.Utils /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the @@ -105,13 +106,13 @@ private[spark] class CompressedMapStatus( MapStatus.decompressSize(compressedSizes(reduceId)) } - override def writeExternal(out: ObjectOutput): Unit = { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) } - override def readExternal(in: ObjectInput): Unit = { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) val len = in.readInt() compressedSizes = new Array[Byte](len) @@ -152,13 +153,13 @@ private[spark] class HighlyCompressedMapStatus private ( } } - override def writeExternal(out: ObjectOutput): Unit = { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) emptyBlocks.writeExternal(out) out.writeLong(avgSize) } - override def readExternal(in: ObjectInput): Unit = { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) emptyBlocks = new RoaringBitmap() emptyBlocks.readExternal(in) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index d49d8fb887007..11c19eeb6e42c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -42,7 +42,7 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long def this() = this(null.asInstanceOf[ByteBuffer], null, null) - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeInt(valueBytes.remaining); Utils.writeByteBuffer(valueBytes, out) @@ -55,7 +55,7 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long out.writeObject(metrics) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { val blen = in.readInt() val byteVal = new Array[Byte](blen) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 554a33ce7f1a6..662a7b91248aa 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -117,11 +117,11 @@ class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { new JavaSerializerInstance(counterReset, classLoader) } - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeInt(counterReset) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { counterReset = in.readInt() } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 142285094342c..259f423c73e6b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -61,13 +61,13 @@ class BlockManagerId private ( def isDriver: Boolean = (executorId == "") - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeUTF(executorId_) out.writeUTF(host_) out.writeInt(port_) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { executorId_ = in.readUTF() host_ = in.readUTF() port_ = in.readInt() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 3db5dd9774ae8..291ddfcc113ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -21,6 +21,8 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import akka.actor.ActorRef +import org.apache.spark.util.Utils + private[spark] object BlockManagerMessages { ////////////////////////////////////////////////////////////////////////////////// // Messages from the master to slaves. @@ -65,7 +67,7 @@ private[spark] object BlockManagerMessages { def this() = this(null, null, null, 0, 0, 0) // For deserialization only - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { blockManagerId.writeExternal(out) out.writeUTF(blockId.name) storageLevel.writeExternal(out) @@ -74,7 +76,7 @@ private[spark] object BlockManagerMessages { out.writeLong(tachyonSize) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { blockManagerId = BlockManagerId(in) blockId = BlockId(in.readUTF()) storageLevel = StorageLevel(in) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 1e35abaab5353..56edc4fe2e4ad 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -97,12 +98,12 @@ class StorageLevel private( ret } - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeByte(toInt) out.writeByte(_replication) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { val flags = in.readByte() _useDisk = (flags & 8) != 0 _useMemory = (flags & 4) != 0 diff --git a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala index 2b452ad33b021..770ff9d5ad6ae 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala @@ -29,7 +29,7 @@ private[spark] class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable { def value = buffer - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val length = in.readInt() buffer = ByteBuffer.allocate(length) var amountRead = 0 @@ -44,7 +44,7 @@ class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable buffer.rewind() // Allow us to read it later } - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { out.writeInt(buffer.limit()) if (Channels.newChannel(out).write(buffer) != buffer.limit()) { throw new IOException("Could not fully write buffer to output stream") 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 65bdbaae65463..e1dc49238733c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -969,6 +969,21 @@ private[spark] object Utils extends Logging { } } + /** + * Execute a block of code that evaluates to Unit, re-throwing any non-fatal uncaught + * exceptions as IOException. This is used when implementing Externalizable and Serializable's + * read and write methods, since Java's serializer will not report non-IOExceptions properly; + * see SPARK-4080 for more context. + */ + def tryOrIOException(block: => Unit) { + try { + block + } catch { + case e: IOException => throw e + case NonFatal(t) => throw new IOException(t) + } + } + /** Default filtering function for finding call sites using `getCallSite`. */ private def coreExclusionFunction(className: String): Boolean = { // A regular expression to match classes of the "core" Spark API that we want to skip when diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 4b2ea45fb81d0..2de2a7926bfd1 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -66,7 +66,7 @@ class SparkFlumeEvent() extends Externalizable { var event : AvroFlumeEvent = new AvroFlumeEvent() /* De-serialize from bytes. */ - def readExternal(in: ObjectInput) { + def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { val bodyLength = in.readInt() val bodyBuff = new Array[Byte](bodyLength) in.readFully(bodyBuff) @@ -93,7 +93,7 @@ class SparkFlumeEvent() extends Externalizable { } /* Serialize to bytes. */ - def writeExternal(out: ObjectOutput) { + def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { val body = event.getBody.array() out.writeInt(body.length) out.write(body) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index b4adf0e9651a8..e59c24adb84af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -22,6 +22,7 @@ import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import org.apache.spark.Logging import org.apache.spark.streaming.scheduler.Job import org.apache.spark.streaming.dstream.{DStream, ReceiverInputDStream, InputDStream} +import org.apache.spark.util.Utils final private[streaming] class DStreamGraph extends Serializable with Logging { @@ -160,7 +161,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { } @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { logDebug("DStreamGraph.writeObject used") this.synchronized { checkpointInProgress = true @@ -172,7 +173,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { } @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug("DStreamGraph.readObject used") this.synchronized { checkpointInProgress = true diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala index 213dff6a76354..7053f47ec69a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -33,6 +33,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Interval, Duration, Time} import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.api.java._ +import org.apache.spark.util.Utils /** @@ -73,13 +74,13 @@ private[python] class TransformFunction(@transient var pfunc: PythonTransformFun pfunc.call(time.milliseconds, rdds) } - private def writeObject(out: ObjectOutputStream): Unit = { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { val bytes = PythonTransformFunctionSerializer.serialize(pfunc) out.writeInt(bytes.length) out.write(bytes) } - private def readObject(in: ObjectInputStream): Unit = { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val length = in.readInt() val bytes = new Array[Byte](length) in.readFully(bytes) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 65f7ccd318684..eabd61d713e0c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.util.{CallSite, MetadataCleaner} +import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -400,7 +400,7 @@ abstract class DStream[T: ClassTag] ( } @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".writeObject used") if (graph != null) { graph.synchronized { @@ -423,7 +423,7 @@ abstract class DStream[T: ClassTag] ( } @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() generatedRDDs = new HashMap[Time, RDD[T]] () diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index f33c0ceafdf42..0dc72790fbdbd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.FileSystem import org.apache.spark.Logging import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils private[streaming] class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) @@ -119,7 +120,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".writeObject used") if (dstream.context.graph != null) { dstream.context.graph.synchronized { @@ -142,7 +143,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() timeToOldestCheckpointFileTime = new HashMap[Time, Time] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 9eecbfaef363f..8152b7542ac57 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.util.TimeStampedHashMap +import org.apache.spark.util.{TimeStampedHashMap, Utils} private[streaming] @@ -151,7 +151,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas } @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() generatedRDDs = new HashMap[Time, RDD[(K,V)]] () diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 9327ff4822699..2154c24abda3a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -73,7 +73,7 @@ class TestOutputStream[T: ClassTag](parent: DStream[T], // This is to clear the output buffer every it is read from a checkpoint @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { ois.defaultReadObject() output.clear() } @@ -95,7 +95,7 @@ class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T], // This is to clear the output buffer every it is read from a checkpoint @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { ois.defaultReadObject() output.clear() } From 898b22ab1fe90e8a3935b19566465046f2256fa6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 Oct 2014 17:21:08 -0700 Subject: [PATCH 018/115] [SPARK-4056] Upgrade snappy-java to 1.1.1.5 This upgrades snappy-java to 1.1.1.5, which improves error messages when attempting to deserialize empty inputs using SnappyInputStream (see https://github.com/xerial/snappy-java/issues/89). Author: Josh Rosen Author: Josh Rosen Closes #2911 from JoshRosen/upgrade-snappy-java and squashes the following commits: adec96c [Josh Rosen] Use snappy-java 1.1.1.5 cc953d6 [Josh Rosen] [SPARK-4056] Upgrade snappy-java to 1.1.1.4 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a1195262614dd..030bea948b5ce 100644 --- a/pom.xml +++ b/pom.xml @@ -346,7 +346,7 @@ org.xerial.snappy snappy-java - 1.1.1.3 + 1.1.1.5 net.jpountz.lz4 From 3a845d3c048eebb0bddb3937128746fde3e8e4d8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 24 Oct 2014 18:36:35 -0700 Subject: [PATCH 019/115] [SQL] Update Hive test harness for Hive 12 and 13 As part of the upgrade I also copy the newest version of the query tests, and whitelist a bunch of new ones that are now passing. Author: Michael Armbrust Closes #2936 from marmbrus/fix13tests and squashes the following commits: d9cbdab [Michael Armbrust] Remove user specific tests 65801cd [Michael Armbrust] style and rat 8f6b09a [Michael Armbrust] Update test harness to work with both Hive 12 and 13. f044843 [Michael Armbrust] Update Hive query tests and golden files to 0.13 --- .rat-excludes | 1 + dev/run-tests | 2 +- project/SparkBuild.scala | 6 +- .../catalyst/analysis/HiveTypeCoercion.scala | 13 + .../catalyst/expressions/complexTypes.scala | 25 + .../org/apache/spark/sql/SQLContext.scala | 1 - .../execution/HiveCompatibilitySuite.scala | 104 +- .../apache/spark/sql/hive/HiveContext.scala | 3 +- .../spark/sql/hive/HiveInspectors.scala | 1 - .../spark/sql/hive/HiveMetastoreCatalog.scala | 1 - .../org/apache/spark/sql/hive/HiveQl.scala | 21 +- .../apache/spark/sql/hive/TableReader.scala | 1 - .../org/apache/spark/sql/hive/TestHive.scala | 7 +- .../sql/hive/api/java/JavaHiveContext.scala | 6 +- .../execution/DescribeHiveTableCommand.scala | 2 +- .../test/resources/data/conf/hive-site.xml | 80 +- .../test/resources/data/files/ProxyAuth.res | 15 + .../test/resources/data/files/alltypes.txt | 2 + .../test/resources/data/files/alltypes2.txt | 2 + .../src/test/resources/data/files/alltypesorc | Bin 0 -> 377237 bytes .../resources/data/files/char_varchar_udf.txt | 1 + .../test/resources/data/files/datatypes.txt | 6 +- .../src/test/resources/data/files/decimal.txt | 18 + .../src/test/resources/data/files/dept.txt | 4 + .../src/test/resources/data/files/emp.txt | 6 + .../data/files/exported_table/_metadata | 1 + .../data/files/exported_table/data/data | 2 + .../ext_test_space/folder+with space/data.txt | 3 + .../data/files/futurama_episodes.avro | Bin 0 -> 3044 bytes .../data/files/header_footer_table_1/0001.txt | 8 + .../data/files/header_footer_table_1/0002.txt | 8 + .../data/files/header_footer_table_1/0003.txt | 4 + .../header_footer_table_2/2012/01/01/0001.txt | 8 + .../header_footer_table_2/2012/01/02/0002.txt | 8 + .../header_footer_table_2/2012/01/03/0003.txt | 4 + .../files/header_footer_table_3/empty1.txt} | 0 .../files/header_footer_table_3/empty2.txt} | 0 .../src/test/resources/data/files/input.txt | 7 + .../test/resources/data/files/keystore.jks | Bin 0 -> 2248 bytes .../src/test/resources/data/files/kv9.txt | 27 + .../src/test/resources/data/files/loc.txt | 8 + .../resources/data/files/non_ascii_tbl.txt | 1 + .../data/files/orc_create_people.txt | 200 +- .../resources/data/files/orc_split_elim.orc | Bin 0 -> 246402 bytes .../resources/data/files/parquet_create.txt | 3 + .../data/files/parquet_partitioned.txt | 3 + .../resources/data/files/parquet_types.txt | 21 + .../test/resources/data/files/person age.txt | 10 +- .../test/resources/data/files/person+age.txt | 3 + .../resources/data/files/posexplode_data.txt | 4 + .../src/test/resources/data/files/sample.json | 1 + .../test/resources/data/files/symlink1.txt | 4 +- .../test/resources/data/files/symlink2.txt | 2 +- .../test/resources/data/files/truststore.jks | Bin 0 -> 958 bytes .../resources/data/scripts/input20_script.py | 30 + ...' + '1'-0-77504a9f3d712143beb52f3c25a904cb | 2 +- .../'1' + 1-0-130514c6116c311d808590a075b187b | 2 +- ...1' + 1.0-0-5db3b55120a19863d96460d399c2d0e | 2 +- ...1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 | 2 +- ...1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 | 2 +- ...1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a | 2 +- .../1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a | 2 +- .../1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 | 2 +- ...1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 | 2 +- .../1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e | 2 +- .../1 + 1S-0-2e99da48f67f588c9e632a57c713522e | 2 +- .../1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f | 2 +- ...0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 | 2 +- ...1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 | 2 +- ...0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d | 2 +- ....0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 | 2 +- ....0 + 1S-0-31fbe14d01fb532176c1689680398368 | 2 +- ....0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 | 2 +- ...L + '1'-0-6e39c7be301f3846efa9b4c939815b4a | 2 +- .../1L + 1-0-1864a260554255a09e4f28b8551eef9d | 2 +- ...L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 | 2 +- ...1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f | 2 +- ...1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 | 2 +- ...1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 | 2 +- ...S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 | 2 +- .../1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 | 2 +- ...S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e | 2 +- ...1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 | 2 +- ...1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e | 2 +- ...1S + 1Y-0-e59bc8279cd364224476ffc504c7685b | 2 +- ...Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb | 2 +- .../1Y + 1-0-a4541db51882b19503649138fbb295f | 2 +- ...Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d | 2 +- ...1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 | 2 +- ...1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 | 2 +- ...1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 | 2 +- ...M_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 | 309 -- ...M_COUNT-0-a393cfc24ad74f930f3284743254c10c | 309 -- ...M_COUNT-0-ae497f1556f548c1e2da9244397a985d | 309 -- ...quences-0-2f25c33d97c43f3276171624d988a286 | 2 +- .../LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 | 2 +- ...h group-0-f52ca483a3e5eadc1b20ba8320d029a7 | 2 +- ...age + 1-0-5e296b921c470f0f0b5d099f28bd5935 | 2 +- ...Average-0-c197ea78c4d8f85f1a317805b6da07e5 | 2 +- ..._exist-11-9c36cac1372650b703400c60dd29042c | 2 +- ..._exist-18-d824f22606f48dfca48ce241a7505f5b | 2 +- ..._exist-20-d824f22606f48dfca48ce241a7505f5b | 2 +- ..._exist-22-d824f22606f48dfca48ce241a7505f5b | 2 +- ..._exist-24-d824f22606f48dfca48ce241a7505f5b | 2 +- ...t_exist-3-d824f22606f48dfca48ce241a7505f5b | 2 +- ...t_exist-5-d824f22606f48dfca48ce241a7505f5b | 2 +- ...t_exist-7-d824f22606f48dfca48ce241a7505f5b | 2 +- ...t_exist-9-d824f22606f48dfca48ce241a7505f5b | 2 +- ...ultiple-1-4d9d4efbabc9fffef8841cc049f479c1 | 29 - ...ultiple-4-7950c676506564b085b41426ed41747c | 2 +- ...itelist-0-3806584ff765bca682594008b90fc304 | 2 +- ...itelist-0-3c23ae800b2f6fb93620890da2dba196 | 2 +- ...as.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 | 2 +- .../alter2-1-aac9c2c7033fd7264c9a107a88ff591 | 10 +- .../alter2-11-aac9c2c7033fd7264c9a107a88ff591 | 10 +- .../alter2-14-aac9c2c7033fd7264c9a107a88ff591 | 10 +- ...alter2-15-75a213649242c2410ea6846f08c91d75 | 2 +- .../alter2-17-aac9c2c7033fd7264c9a107a88ff591 | 10 +- ...alter2-18-75a213649242c2410ea6846f08c91d75 | 2 +- ...alter2-20-9c36cac1372650b703400c60dd29042c | 2 +- ...alter2-25-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-28-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-29-63f13c364546ddce5d2176c6604a948f | 2 +- ...alter2-31-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-32-63f13c364546ddce5d2176c6604a948f | 2 +- ...alter2-35-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-38-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-39-63f13c364546ddce5d2176c6604a948f | 2 +- .../alter2-4-aac9c2c7033fd7264c9a107a88ff591 | 10 +- ...alter2-41-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-42-63f13c364546ddce5d2176c6604a948f | 2 +- .../alter2-5-75a213649242c2410ea6846f08c91d75 | 2 +- .../alter2-7-aac9c2c7033fd7264c9a107a88ff591 | 10 +- .../alter2-8-75a213649242c2410ea6846f08c91d75 | 2 +- ...alter3-1-47f70392b97b94cdb425b25bde204f58} | 0 ...alter3-12-2fcb7fc251f682a584ad513fddfac506 | 12 +- ...alter3-16-9c36cac1372650b703400c60dd29042c | 2 +- ...lter3-21-91e32b3028ecc352dad8884829148311} | 0 ...alter3-25-568a59760e5d3241b63d65cce595face | 2 +- ...alter3-27-54ad133b447f67c6d1ed7d4c43803a87 | 12 +- ...alter3-28-5332228ea451105c897d0c8c3c8f2773 | 12 +- ...alter3-29-b8fba19b9009131caffbb5fe7468b67c | 2 +- ...alter3-32-327744965ee8ed630f56fa3e4a3c5c65 | 12 +- .../alter3-5-bf2a8fd1884bb584059c848332e30c97 | 2 +- .../alter3-7-30be5698ca15c1fd836686e7ad48ad8 | 12 +- .../alter3-8-8f0a466bd1d021e40690865b7ae52a43 | 12 +- .../alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 | 2 +- .../alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 | 6 +- ...alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 | 6 +- ...alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 | 6 +- ...alter4-16-549981e00a3d95f03dd5a9ef6044aa20 | 2 +- .../alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 | 6 +- .../alter4-5-9c36cac1372650b703400c60dd29042c | 2 +- ...alter5-1-cbad54fbd4a08fc8717708f93358ec3e} | 0 ...lter5-15-cbad54fbd4a08fc8717708f93358ec3e} | 0 ...alter5-18-2a9c8219c1468a1cf0534c665d1fcebf | 8 +- ...alter5-21-2a9c8219c1468a1cf0534c665d1fcebf | 8 +- .../alter5-4-2a9c8219c1468a1cf0534c665d1fcebf | 8 +- .../alter5-7-2a9c8219c1468a1cf0534c665d1fcebf | 8 +- ...er_index-2-f36cb2eed39691ca949b25182e2dd31 | 4 +- ...er_index-4-f36cb2eed39691ca949b25182e2dd31 | 4 +- ...r_index-6-489b4ceb2f4301a7132628303f99240d | 2 +- ...erge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 | 2 +- ...merge_2-2-bf243aa10b608872b9e8286f89c5ff30 | 14 +- ...erge_2-3-bc9bb363f9a2026cfc70a31bb4551352} | 0 ...erge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224} | 0 ...erge_2-5-6319bf26f3739260b1a77e2ea89ef147} | 0 ...merge_2-6-f2eeb518a957cece4250cc7558839e02 | 2 +- ...merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 | 2 +- ...merge_2-9-f2eeb518a957cece4250cc7558839e02 | 2 +- ...oltype-12-84807e0be3e91250d4b8f2dc7938a256 | 1 - ...oltype-14-fdad866a990591083719bd45e4113f58 | 1 - ...oltype-16-b0534980e325d1fee63c2636ad3f8a4e | 75 - ...oltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c | 1 - ...oltype-19-63799ee74ccc42d9bb817e6d00a6fae3 | 1 - ..._coltype-2-17e04afbb81a724fe8c47c07b642f9a | 10 - ...coltype-21-17e04afbb81a724fe8c47c07b642f9a | 10 - ...oltype-22-639cdccdea5f4b1863f9b17b04183c93 | 10 - ...oltype-23-325be9a5d7c0277498a139c0a9fdb26a | 10 - ...oltype-24-71de9e476503e83f82baf1ab17da87f6 | 10 - ...coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 | 1 - ...coltype-6-db84ad9022cdc1828c24a0340321c8fd | 1 - ...coltype-8-42a70ae131fbb834c79074fdbd7beea0 | 1 - ...mat_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 | 6 +- ...at_loc-11-fe39b84ddc86b6bf042dc30c1b612321 | 10 +- ...at_loc-13-fe39b84ddc86b6bf042dc30c1b612321 | 10 +- ...at_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 | 10 +- ...at_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 | 10 +- ...at_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 | 10 +- ...mat_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 | 6 +- ...mat_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 | 6 +- ...mat_loc-9-fe39b84ddc86b6bf042dc30c1b612321 | 10 +- ...t_mode-1-e11f1476d2f30ab7a83f95013809f9e6} | 0 ...t_mode-2-29b4b67965f24c40bcc426043efb892d} | 0 ...t_mode-3-2100fc7853625d8c7dad5c0ee70d4690} | 0 ...t_mode-4-c3fa911536c75e87febf2874e7889879} | 0 ...ct_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 | 2 +- ...ct_mode-9-d1b12be1d01eabaf244f41e74d902d9d | 2 +- ...itelist-0-3c23ae800b2f6fb93620890da2dba196 | 2 +- ...tition-11-bc84e38ccad173f5b47474d91db244d7 | 2 +- ...tition-14-9c36cac1372650b703400c60dd29042c | 2 +- ...ition-19-d862c00e5a37ae841a6c8ec6c5d68e8c} | 0 ...rtition-2-9c36cac1372650b703400c60dd29042c | 2 +- ...tition-22-d50111b57d14f1ded1c47c773b0e0ac2 | 2 +- ...tition-24-21dd05d56ebba285a8eb5bde5904d6a3 | 2 +- ...tition-26-270655c514bb6f04acd4459df52dd77b | 2 +- ...tition-4-833254c8f7c5b1248e1a249560652627} | 0 ...rtition-7-e3d9a36d53d30de215b855095c58d0d7 | 2 +- ...rtition-9-21dd05d56ebba285a8eb5bde5904d6a3 | 2 +- ...e_serde-1-5bc931a540f0fec54e852ff10f52f879 | 8 +- ..._serde-11-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ..._serde-13-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ..._serde-15-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ..._serde-17-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ...e_serde-3-5bc931a540f0fec54e852ff10f52f879 | 8 +- ...e_serde-5-5bc931a540f0fec54e852ff10f52f879 | 8 +- ...e_serde-9-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ...rchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb} | 0 ...archar2-3-fb3191f771e2396d5fc80659a8c68797 | 2 +- ...archar2-5-84e700f9dc6033c1f237fcdb95e31a0c | 2 +- ...rchar2-6-3250407f20f3766c18f44b8bfae1829d} | 0 ...rchar2-7-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...archar2-7-84e700f9dc6033c1f237fcdb95e31a0c | 1 - ...archar2-8-4c12c4c53d99338796be34e603dc612c | 1 - ...archar2-8-84e700f9dc6033c1f237fcdb95e31a0c | 1 + ...archar2-9-4c12c4c53d99338796be34e603dc612c | 1 + ..._select-2-1ac845048a8c714a36a719ea8e4f570b | 12 +- ..._select-4-1ac845048a8c714a36a719ea8e4f570b | 6 +- ..._select-6-1ac845048a8c714a36a719ea8e4f570b | 8 +- ...ew_rename-2-67e47ee2746463594d5c48b10ba1bb | 5 - ..._rename-4-19c1c00f0aa99d81b7466958c15d88e3 | 5 - ...us_col-0-e91e3e5a22029b9b979ccbbef97add66} | 0 ...us_col-1-b4fe82679efdf6a15e9ecff53baf8d8d} | 0 ...us_col-2-dadfa3854356dead14b93c5a71a5d8ab} | 0 ...us_col-3-70509ccd2765d90b98666b6dff8afe1b} | 0 ...s_join-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ...s_join-1-84e7846d50fc15e836c83911ce039871} | 0 ..._join-10-a59dc1c01d48c82d46a5beab759f618d} | 0 ..._join-11-3e0ade2476221f6396381f55a3b82e60} | 0 ..._join-12-cef44682d339a67ba765f854da21f976} | 0 ..._join-13-6292c27f0c4eaeab2a5e6c317e3afa2e} | 0 ..._join-14-4f3042b9feebd00c540ddac40e7254d1} | 0 ..._join-15-a2f3b8a636e46e5df514df46c452855f} | 0 ..._join-16-a75699a21ea36d962a1ba45bd5a12f26} | 0 ..._join-17-64380f94a431e3a1698aa2edd3d0a6b2} | 0 ..._join-18-222d1fcce7420950436e00d3a1bba957} | 0 ..._join-19-dea5f8f5c703583c7a3bdcb62cd3d589} | 0 ...s_join-2-40548ec2313af8dbdcbb8ad0477d8600} | 0 ..._join-20-2d5e186b97166e3a625a169d0d73f9c8} | 0 ..._join-21-ed73d4b01424287148347ccf721b37e2} | 0 ..._join-22-2cf93da6bb0efdafeaa989c9df236701} | 0 ..._join-23-fa90806f6137300311df032e28df3d4c} | 0 ..._join-24-4a6976344eeae35e059285ed78f9feb3} | 0 ...s_join-3-26f82fb6734d5bc6f7159b06c0949178} | 0 ...s_join-4-a598c93d86a646cfa6ea86da54bce9b8} | 0 ...s_join-5-d12ba848d20d1def95379c490068f050} | 0 ...s_join-6-96a4806e61c5d882affac57e115f285f} | 0 ...s_join-7-38be41d6328f57350a78fb2097d1ebd2} | 0 ...s_join-8-eb11e867a36e2114ef0f0f001e01984c} | 0 ...s_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd} | 0 ..._limit-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ..._limit-1-26f82fb6734d5bc6f7159b06c0949178} | 0 ...limit-10-a89c94fd0de0cfc96725fea890987cc0} | 0 ..._limit-2-eb11e867a36e2114ef0f0f001e01984c} | 0 ..._limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd} | 0 ..._limit-4-4252006172a476fbc591eebee49bffa3} | 0 ..._limit-5-a2f3b8a636e46e5df514df46c452855f} | 0 ..._limit-6-f1fd6c403a9787947877f48c772afe96} | 0 ..._limit-7-69b422026fce4e2d2cde9a478d6aaa40} | 0 ..._limit-8-72f5e55c1e244084eea255c32a6f58eb} | 0 ..._limit-9-9da67c62d5e3145d450ad7953b33361f} | 0 ...s_part-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ...ts_part-1-3c29684bfd2df7439ee0551eb42cfa0} | 0 ..._part-10-5ba0295bfe42deb678e59b3a330b14ff} | 0 ...ts_part-11-dbdbe2e04c5dad6c8af78b6386b329} | 0 ..._part-12-2856fb4f4af8932804cb238429d9de6f} | 0 ..._part-13-4fa8b0f9fb0202ac0e82fb87538d6445} | 0 ..._part-14-62c557bfb7bf70694a32ebceea329ebd} | 0 ..._part-15-f796cd035603726a5c4ce3e71194822b} | 0 ..._part-16-45eb5544124364714549f199f9c2b2ac} | 0 ..._part-17-ad61ebd912b6bef3778c4ff38c0be5ab} | 0 ..._part-18-b9f2dff7c7b57412cea44433ea810fa7} | 0 ..._part-19-10cab43c2966718bb39e2f22365cd6c1} | 0 ...s_part-2-a4fb8359a2179ec70777aad6366071b7} | 0 ..._part-20-51ec5046b50543899ed54c9fc2b885af} | 0 ..._part-21-d69cb7b7770b51bc4b99d5d0f74d4662} | 0 ..._part-22-22f430881fbfe92736988c2a4184fe0c} | 0 ..._part-23-2719ee3f0b8b07f1e5c80f8329b9f87f} | 0 ..._part-24-1f7bdd816b83f45a6d53d08b6866109f} | 0 ..._part-25-d1599e385e8bad6cd62d564a6fd0536f} | 0 ..._part-26-ec26bcb2587f4638b364ba494ac373e0} | 0 ..._part-27-7804e2e230a42a46d72546fa2381b423} | 0 ..._part-28-40f2a1f06d181ef93edf42a591cbf15e} | 0 ...s_part-3-16367c381d4b189b3640c92511244bfe} | 0 ...s_part-4-397e834a25a20b68aed5e87d55ea6174} | 0 ...s_part-5-cef44682d339a67ba765f854da21f976} | 0 ...s_part-6-f4263aca1612c05cb08242448191ad05} | 0 ...s_part-7-b222533ddda09918c4f5acc408bf1a02} | 0 ...s_part-8-84e3979946de67779a9704a3adc2184f} | 0 ...ts_part-9-c6c38e93babafe56cd4f177a17d37b8} | 0 ..._table-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ...s_table-1-3c29684bfd2df7439ee0551eb42cfa0} | 0 ...table-10-a7419af512a6c0b60ec51e556608cdad} | 0 ...table-11-4be11140c2e21b3a172224d103844785} | 0 ...table-12-84bc7cf67544461e9102820ed410dfb6} | 0 ...s_table-13-d2acfdf279aad035b31ed61e87bff6} | 0 ...table-14-60d1c44b8412c4983265e320ecfb25f0} | 0 ...table-15-7b185296b6a0c6ceff2df1f89fecc6b4} | 0 ...table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d} | 0 ...table-17-cd84d629d569a09d139f38dae01e677c} | 0 ...table-18-7012af4092c18075f4f3a00b80f491ee} | 0 ..._table-2-84e7846d50fc15e836c83911ce039871} | 0 ..._table-3-a598c93d86a646cfa6ea86da54bce9b8} | 0 ..._table-4-d12ba848d20d1def95379c490068f050} | 0 ..._table-5-4a8ad142e57c5dce2623e92f5e5ad15a} | 0 ..._table-6-a59dc1c01d48c82d46a5beab759f618d} | 0 ..._table-7-6292c27f0c4eaeab2a5e6c317e3afa2e} | 0 ..._table-8-43cb040e2fe01904bc52e198fcae9b3d} | 0 ..._table-9-eadfdc61d22bd22bbf5a69370908a82e} | 0 ..._union-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ..._union-1-26f82fb6734d5bc6f7159b06c0949178} | 0 ...union-10-3aa3eb68d092ea0157c5426a5f2ca3f9} | 0 ...union-11-c1302094d7b4456550826535b529468b} | 0 ...union-12-3af760f960a2e0995d1bc4afef0c5aa0} | 0 ...union-13-26f82fb6734d5bc6f7159b06c0949178} | 0 ...union-14-eb11e867a36e2114ef0f0f001e01984c} | 0 ...union-15-e0b7cb2226234fbea8ae11ea7d9d84bd} | 0 ...union-16-4252006172a476fbc591eebee49bffa3} | 0 ...union-17-a2f3b8a636e46e5df514df46c452855f} | 0 ...union-18-58d55f8c8b2489726232a00254499144} | 0 ...union-19-70c851c23a7ac89f0366b9c26085cc3e} | 0 ..._union-2-eb11e867a36e2114ef0f0f001e01984c} | 0 ...union-20-222d1fcce7420950436e00d3a1bba957} | 0 ...union-21-a5503df8f50e057415719bcfe4c69f13} | 0 ...union-22-a23089e5a76dc35df04d9936d60c4565} | 0 ..._union-3-e0b7cb2226234fbea8ae11ea7d9d84bd} | 0 ..._union-4-4252006172a476fbc591eebee49bffa3} | 0 ..._union-5-a2f3b8a636e46e5df514df46c452855f} | 0 ..._union-6-222d1fcce7420950436e00d3a1bba957} | 0 ..._union-7-aab1a7b7e1fe4d061a580126d67dfd0a} | 0 ..._union-8-172e0d089bd5bcbaf54775a618d826bb} | 0 ..._union-9-69b422026fce4e2d2cde9a478d6aaa40} | 0 ...archive-0-89cd75b0565e8d96910d5528db9984e7 | 1 - ...archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ...rchive-11-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...rchive-13-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...archive-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 - ...rchive-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 - ...rchive-17-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...rchive-20-530277b0fee8b05c37b26846bceef827 | 6 - ...rchive-22-530277b0fee8b05c37b26846bceef827 | 6 - ...rchive-24-530277b0fee8b05c37b26846bceef827 | 6 - ...rchive-28-188eb7912265ed8dffa5200517bbe526 | 1 - ...rchive-30-bea4ae5a0d219d544ea0b53bf29ecc7a | 1 - ...adoop20-0-89cd75b0565e8d96910d5528db9984e7 | 1 - ...adoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ...doop20-11-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...doop20-13-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...adoop20-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 - ...doop20-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 - ...doop20-17-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...doop20-20-530277b0fee8b05c37b26846bceef827 | 6 - ...doop20-22-530277b0fee8b05c37b26846bceef827 | 6 - ...doop20-24-530277b0fee8b05c37b26846bceef827 | 6 - ...doop20-28-188eb7912265ed8dffa5200517bbe526 | 1 - ...doop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a | 1 - ...e_multi-0-89cd75b0565e8d96910d5528db9984e7 | 1 - ...e_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ..._multi-11-cf5431cd843666b95ad2a82b334ac01e | 1 - ..._multi-13-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...e_multi-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 - ..._multi-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 - ..._multi-17-27895cbe0ee6d24d7fc866314491e1bb | 1 - .../attr-0-24e06ffd262f2a5a6eec3314445d83ba | 2 +- ...o_join0-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join0-2-7bd04899197b027d81c24e45a99ad15c | 2 +- ...o_join1-0-443afb71720bad780b5dbfb6dbf4b51a | 2 +- ...to_join1-4-ae1247a065c41ce0329ca6078ab586e | 2 +- ..._join10-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join10-2-eef4ee52e0783b15fb5fe17378806b13 | 2 +- ..._join11-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join11-2-5496e81f60ba1d8a95d8375589c71e05 | 2 +- ..._join12-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join12-2-4df549c5f0b6bff0c843008fa35b1320 | 2 +- ..._join13-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join13-2-5ff417533a1243cd6fc556960fa170c9 | 2 +- ..._join14-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join14-2-9b141c1e5917ca82c6bc36a9a2950a1e | 2 +- ...o_join14-3-2b9ccaa793eae0e73bf76335d3d6880 | 2 +- ..._join14-4-bab89dfffa77258e34a595e0e79986e3 | 2 +- ...o_join14-7-5b5ded1412301eae5f8f705a39e6832 | 2 +- ..._join15-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 | 2 +- ..._join16-0-ce1ef910fff98f174931cc641f7cef3a | 1 - ..._join16-2-66e56dcda38eb09819ac49e47e40d125 | 1 - ..._join17-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join17-4-11d706a64d44a8b0d41b290c4671c29c | 2 +- ..._join18-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join18-2-f633ade9577c8b0e89d89124194c8d0f | 2 +- ..._join19-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join19-4-eaa70da463b92e85e1796277f016c18f | 2 +- ...o_join2-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join2-4-9d8144612cb3132ad9f7c8fa93586185 | 2 +- ..._join20-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join20-2-903ee25e327188edaaf2040fec5a8e52 | 2 +- ..._join20-4-e48e08d5b94719d56a58284eaef757f2 | 2 +- ..._join21-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join21-2-3536b7d78713e86ee67f5f6c9b88958f | 2 +- ..._join22-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join22-2-a4abc288c20edee53ede45d248cf3abb | 2 +- ..._join23-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join23-2-6d2c5b58222f31658a0cf957e093a150 | 2 +- ..._join24-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join24-4-d79325ef6494aa87843fdfd78de7c812 | 2 +- ..._join26-1-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join26-4-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- ..._join27-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join27-2-fceaa1ebd63334061d2d8daf961e935e | 2 +- ..._join28-0-10a2c01dccc8980fe6aff1f9dd65042c | 2 +- ..._join28-1-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join3-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join3-4-eaa70da463b92e85e1796277f016c18f | 2 +- ..._join30-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...join30-10-820f36ed1bdf14c1deb383f508a5ed7a | 2 +- ..._join30-12-de6e26f52f6f9ea5ef9a4868e57d36d | 2 +- ...join30-14-7a248488c218919ab50e072fdbdecb73 | 2 +- ...join30-16-f4f5bc179d84baf57e14cd2f8bd39436 | 2 +- ..._join30-2-bc472f95600f47d5ea60fdeddc59dbc7 | 2 +- ...to_join30-4-f5083eca9c3df277988d8b345b8d43 | 2 +- ..._join30-6-4a9144326fc7d066c9aadb13d1b95031 | 2 +- ..._join30-8-8a27209399df7f9c4d15988b11753a61 | 2 +- ..._join31-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 | 2 +- ..._join32-0-24ca942f094b14b92086305cc125e833 | 2 +- ...oin32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c} | 0 ...oin32-13-ee2dcaae78ae900ffce8d19fbadc3735} | 0 ...oin32-14-7927c2ce644d1ce1de251405c8563e99} | 0 ...join32-15-4cd3b51861720ac06c6deb818c83670} | 0 ...oin32-20-693874ea8e06a8b155394ac27af2b1a7} | 0 ...oin32-21-bf8a1bb0baaae9fbf1c3aa656f991f42} | 0 ...oin32-22-3d14d63e996851f51a98f987995d8da6} | 0 ...oin32-23-8b183ec2c164b3b530e802ffc880a5fa} | 0 ..._join32-5-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ...o_join32-6-442e69416faaea9309bb8c2a3eb73ef | 2 +- ..._join32-7-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...o_join4-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join4-4-998c3a307b074a6505bb7fcef276be04 | 2 +- ...o_join5-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join5-4-998c3a307b074a6505bb7fcef276be04 | 2 +- ...o_join6-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join6-4-998c3a307b074a6505bb7fcef276be04 | 2 +- ...o_join7-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join7-4-30d0c1a49784347fedbac21a69c3a899 | 2 +- ...o_join9-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join9-4-eaa70da463b92e85e1796277f016c18f | 2 +- ...ilters-2-bee6095f42de6a16708c2f9addc1b9bd} | 0 ...lters-31-268d8fb3cb9b04eb269fe7ec40a24dfe} | 0 ...lters-32-6dc6866a65c74d69538b776b41b06c16} | 0 ...lters-33-e884480a0f7273d3e2f2de2ba46b855c} | 0 ...lters-34-98fd86aea9cacaa82d43c7468109dd33} | 0 ...n_nulls-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 | 2 +- ..._nulls-11-141c550a314d15c4e200e5baeb246de2 | 2 +- ..._nulls-12-8a65225a88da0169af26848c06cb981c | 2 +- ..._nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 | 2 +- ..._nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 | 2 +- ..._nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 | 2 +- ..._nulls-16-dbe244d2c21e477c3703c4ce1903e8af | 2 +- ..._nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 | 2 +- ..._nulls-18-439a409bc50dfd86dee78c151c3de5eb | 2 +- ..._nulls-19-92641e46934ebbf3d44e6e60de1882f4 | 2 +- ..._nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f} | 0 ..._nulls-20-e34b2b210059a5f93c0a873d91859b5d | 2 +- ..._nulls-21-c7aaa831acbd959c6d1793056e3c288a | 2 +- ..._nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 | 2 +- ..._nulls-23-5805a41289a26139c06604a40bf5a6fa | 2 +- ..._nulls-24-80991af26d5d37e0864ecc2c8ab0b984 | 2 +- ...n_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 | 2 +- ...in_nulls-4-fc1128c86cd430db8cd4ff834be4562 | 2 +- ...n_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc | 2 +- ...n_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 | 2 +- ...n_nulls-7-8395fa78507105c2a018e88f717b95e2 | 2 +- ...n_nulls-8-fd992f2127a139aeb554d797e748ed54 | 2 +- ...n_nulls-9-e3a86622a437e910b7225d1e6108da9e | 2 +- ...values-1-2bfb628930d072124636d21d82e3b462} | 0 ...values-4-11af6838bb9e04152c2f9a7e2044abe0} | 0 ...values-7-99fcaa5203ed3debb52c9086028dc8c2} | 0 ..._values-8-950af86c321a67ab3ed0fa5b63ea6aed | 2 +- ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_14-10-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_14-12-94538bc7322522a5534cafc0551d2189 | 2 +- ...oin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 | 2 +- ...oin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 | 2 +- ...oin_14-18-21269869cd3aaf4ade2170d9017de018 | 2 +- ...pjoin_14-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...oin_14-20-4e0e8cd0626a84b21ca7d2f633623578 | 2 +- ...oin_14-22-2fe7b834b341bf18e36cd79dd00ec16a | 2 +- ...oin_14-24-43ba2c72db9db1ec18d835ec978f8da1 | 2 +- ...oin_14-26-b66c416fdd98d76981f19e9c14b6a562 | 2 +- ...oin_14-28-b889b147255231f7fe44bd57e1f8ba66 | 2 +- ...oin_14-30-b9d66e78b8898a97a42d1118300fa0ce | 2 +- ...oin_14-32-b0ca9e20cd48457e6cf1c313d5505213 | 2 +- ...oin_14-37-a45927057c01fd54818b5dd50e77f60e | 2 +- ...oin_14-38-a988727daa49fb5e190f81c027bb7005 | 2 +- ...oin_14-43-a45927057c01fd54818b5dd50e77f60e | 2 +- ...oin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 | 2 +- ...join_14-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_14-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_14-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_1-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...oin_1-10-5cba470fbd02e730781a3b63fd9aa3e2} | 0 ...oin_1-11-337e909605c780d00ad8895686defa06} | 0 ...join_1-12-24ca942f094b14b92086305cc125e833 | 2 +- ...join_1-13-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_1-14-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_1-15-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_1-16-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc | 2 +- ...join_1-19-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_1-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_1-21-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_1-22-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_1-24-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_1-4-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...join_1-5-d964114ed76536c8e3cacd231340851c} | 0 ...join_1-6-e44aff8a860cf3965752d3e1ce725cde} | 0 ...join_1-7-ae582a255a02d91674aab500aee79e20} | 0 ..._join_1-8-962264967269db1d5f28a9a6c60dbf1} | 0 ...join_1-9-10b03ce2526bf180faaec9310cfab290} | 0 ...join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_10-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_10-10-b89ea2173180c8ae423d856f943e061f | 2 +- ...oin_10-11-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_10-12-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...oin_10-14-95e18bd00f2de246efca1756681c1e87 | 2 +- ...join_10-16-caa641c820fcc5f601758c5f0385b4e | 2 +- ..._join_10-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_10-7-24ca942f094b14b92086305cc125e833 | 2 +- ...join_10-8-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_10-9-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_11-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...in_11-10-5cba470fbd02e730781a3b63fd9aa3e2} | 0 ...in_11-11-337e909605c780d00ad8895686defa06} | 0 ...oin_11-12-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_11-14-325432a220aa3ebe8b816069916924d8 | 2 +- ...oin_11-15-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_11-16-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ...join_11-17-442e69416faaea9309bb8c2a3eb73ef | 2 +- ...oin_11-19-325432a220aa3ebe8b816069916924d8 | 2 +- ...oin_11-2-f42438f3f5c266b997686ba846420ebe} | 0 ...oin_11-21-c4d55c247c9326f474d89b29b81d60aa | 2 +- ...oin_11-4-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...oin_11-5-d964114ed76536c8e3cacd231340851c} | 0 ...oin_11-6-e44aff8a860cf3965752d3e1ce725cde} | 0 ...oin_11-7-ae582a255a02d91674aab500aee79e20} | 0 ...join_11-8-962264967269db1d5f28a9a6c60dbf1} | 0 ...oin_11-9-10b03ce2526bf180faaec9310cfab290} | 0 ...oin_12-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...in_12-10-5cba470fbd02e730781a3b63fd9aa3e2} | 0 ...in_12-11-337e909605c780d00ad8895686defa06} | 0 ...oin_12-12-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_12-13-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_12-14-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...oin_12-15-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...in_12-17-97ba394ab6aad2547f399ebbf757a4b6} | 0 ...in_12-18-73ee2d7b62e2aede20ca5de577cd7b7f} | 0 ...oin_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f} | 0 ...oin_12-2-f42438f3f5c266b997686ba846420ebe} | 0 ...oin_12-21-4ecd65f0e26e981b66770b3e91e128fc | 2 +- ...oin_12-4-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...oin_12-5-d964114ed76536c8e3cacd231340851c} | 0 ...oin_12-6-e44aff8a860cf3965752d3e1ce725cde} | 0 ...oin_12-7-ae582a255a02d91674aab500aee79e20} | 0 ...join_12-8-962264967269db1d5f28a9a6c60dbf1} | 0 ...oin_12-9-10b03ce2526bf180faaec9310cfab290} | 0 ...join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_13-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_13-10-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...oin_13-11-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...oin_13-12-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_13-13-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...oin_13-14-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_13-15-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 | 2 +- ...oin_13-19-4b2ac2865384fbca7f374191d8021d51 | 2 +- ..._join_13-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...oin_13-20-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...oin_13-21-ea23403b9eb55e8b06d1c198e439569f | 2 +- ...oin_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 | 2 +- ...oin_13-25-4b2ac2865384fbca7f374191d8021d51 | 2 +- ...oin_13-26-f135547e33c01d1f543c8b1349d60348 | 2 +- ...oin_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 | 2 +- ...oin_13-30-4b2ac2865384fbca7f374191d8021d51 | 2 +- ...join_13-9-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_14-10-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_14-11-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...oin_14-12-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_14-13-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_14-15-43ad2152b18d711adbdd1aeb940b662a | 2 +- ...oin_14-18-a16ff76d72ad428fb9d43ab910f259fd | 2 +- ..._join_14-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_14-7-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_14-8-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_14-9-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_15-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_15-10-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_15-11-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...oin_15-12-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_15-13-24ca942f094b14b92086305cc125e833 | 2 +- ..._join_15-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_15-7-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_15-8-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_15-9-f135547e33c01d1f543c8b1349d60348 | 2 +- ...oin_16-0-24ca942f094b14b92086305cc125e833} | 0 ...oin_16-1-16367c381d4b189b3640c92511244bfe} | 0 ...in_16-10-4706d21b17f993cc1cc94be6b7e04c28} | 0 ...in_16-11-5e81d0b41cc58d8cc151046c7a111411} | 0 ...in_16-12-e8a77ff790bfd6740489fc4374ec6c3d} | 0 ...in_16-13-920818d557d2525dabb5c539a80a1bbb} | 0 ...in_16-14-455dfeeba27ecf53923db0cbf0aab908} | 0 ...in_16-15-11f98f575685beedc14a88fc47a61041} | 0 ...oin_16-16-dff02d7b4c5242434d5e7449bdb67f8b | 24 + ...oin_16-2-7b4ad215fc2e75c71c6614a2b6322e8e} | 0 ...oin_16-3-365488a703b0640acda73a7d7e6efa06} | 0 ...oin_16-4-d0ec6d66ff349db09fd455eec149efdb} | 0 ...oin_16-5-3b0f76816be2c1b18a2058027a19bc9f} | 0 ...oin_16-6-86473a0498e4361e4db0b4a22f2e8571} | 0 ...oin_16-7-7e87a31677022b6a1c360922ef74754e} | 0 ...join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c} | 0 ...join_16-9-ae5880516ea2f924cfbaeb919adc86e} | 0 ...join_2-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...join_2-10-24ca942f094b14b92086305cc125e833 | 2 +- ...join_2-11-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_2-12-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_2-13-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_2-14-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 | 2 +- ...join_2-17-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_2-18-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 2 +- ...join_2-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_2-20-8180638a57b64557e02815c863031755 | 2 +- ...join_2-22-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_2-3-aa6ec7f17b48cf49f02ab7367453ab39} | 0 ...join_2-4-66b07c93d79ed9958b8427dad16c3ef3} | 0 ...join_2-6-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...join_2-7-d964114ed76536c8e3cacd231340851c} | 0 ..._join_2-8-962264967269db1d5f28a9a6c60dbf1} | 0 ...join_2-9-10b03ce2526bf180faaec9310cfab290} | 0 ...join_3-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...join_3-10-24ca942f094b14b92086305cc125e833 | 2 +- ...join_3-11-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_3-12-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_3-13-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_3-14-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc | 2 +- ...join_3-17-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_3-19-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_3-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_3-20-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_3-22-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1} | 0 ...join_3-4-45b63361c1e5178b69a1531b238c8460} | 0 ...join_3-6-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...join_3-7-d964114ed76536c8e3cacd231340851c} | 0 ...join_3-8-e44aff8a860cf3965752d3e1ce725cde} | 0 ...join_3-9-ae582a255a02d91674aab500aee79e20} | 0 ...join_4-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...oin_4-10-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...oin_4-11-d964114ed76536c8e3cacd231340851c} | 0 ...join_4-12-24ca942f094b14b92086305cc125e833 | 2 +- ...join_4-13-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_4-14-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_4-15-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_4-16-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 | 2 +- ...join_4-19-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_4-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_4-21-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_4-22-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_4-24-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_4-3-aa6ec7f17b48cf49f02ab7367453ab39} | 0 ...join_4-4-66b07c93d79ed9958b8427dad16c3ef3} | 0 ...join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1} | 0 ...join_4-6-45b63361c1e5178b69a1531b238c8460} | 0 ...join_4-7-ecca12a2f377c18c53563a534e9dd5a5} | 0 ...join_4-8-65930e1b01da720cf296ca3df668b58d} | 0 ...join_5-1-fdbb11a2de2777dfdd916b59764d5c8e} | 0 ...join_5-10-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_5-11-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 | 2 +- ...join_5-14-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_5-16-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_5-17-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_5-18-24ca942f094b14b92086305cc125e833 | 2 +- ...join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed} | 0 ...join_5-20-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_5-3-9878d6ab0fe143611c88ffc0602197e3} | 0 ...join_5-4-a576657b9fd1c23aed85a7409887b2fb} | 0 ...join_5-6-350b202868590b5edaed18007fd5cbbe} | 0 ...join_5-7-1155788c7c133a73c3609e8052accfa5} | 0 ..._join_5-8-d0ec6d66ff349db09fd455eec149efdb | 2 +- ..._join_5-9-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._join_6-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...join_6-11-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_6-12-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_6-13-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_6-14-24ca942f094b14b92086305cc125e833 | 2 +- ...join_6-15-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...join_6-16-ea23403b9eb55e8b06d1c198e439569f | 2 +- ...join_6-17-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_6-19-cf41f7ce9478536e823107d1810ff1d7 | 2 +- ...e_join_6-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_6-21-b55506a213ec710004e6d7f3462834d0 | 2 +- ...join_6-23-4281442c87dcf6007f8bd42504eba186 | 2 +- ...join_6-25-52f0e65724d29e2b4054b59a50d2837b | 2 +- ...join_6-27-961f141836f2cc9521f681cadbc3d140 | 2 +- ...join_6-29-fd0cc412e0987569a4ed879454b53fb0 | 2 +- ...join_6-30-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_6-32-cf41f7ce9478536e823107d1810ff1d7 | 2 +- ...join_6-34-52f0e65724d29e2b4054b59a50d2837b | 2 +- ...join_6-36-961f141836f2cc9521f681cadbc3d140 | 2 +- ...join_6-38-fd0cc412e0987569a4ed879454b53fb0 | 2 +- ...join_7-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...oin_7-10-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...oin_7-11-d964114ed76536c8e3cacd231340851c} | 0 ...join_7-12-962264967269db1d5f28a9a6c60dbf1} | 0 ...oin_7-13-10b03ce2526bf180faaec9310cfab290} | 0 ...join_7-14-24ca942f094b14b92086305cc125e833 | 2 +- ...join_7-15-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_7-16-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_7-17-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_7-18-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 | 2 +- ...join_7-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_7-21-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_7-23-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_7-24-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_7-26-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_7-3-aa6ec7f17b48cf49f02ab7367453ab39} | 0 ...join_7-4-66b07c93d79ed9958b8427dad16c3ef3} | 0 ...join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1} | 0 ...join_7-6-45b63361c1e5178b69a1531b238c8460} | 0 ...join_7-7-ecca12a2f377c18c53563a534e9dd5a5} | 0 ...join_7-8-65930e1b01da720cf296ca3df668b58d} | 0 ...join_8-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...join_8-10-962264967269db1d5f28a9a6c60dbf1} | 0 ...oin_8-11-10b03ce2526bf180faaec9310cfab290} | 0 ...oin_8-12-5cba470fbd02e730781a3b63fd9aa3e2} | 0 ...oin_8-13-337e909605c780d00ad8895686defa06} | 0 ...join_8-14-24ca942f094b14b92086305cc125e833 | 2 +- ...join_8-15-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_8-16-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_8-17-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_8-18-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc | 2 +- ...join_8-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_8-21-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_8-23-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_8-24-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 2 +- ...join_8-26-8180638a57b64557e02815c863031755 | 2 +- ...join_8-28-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1} | 0 ...join_8-4-45b63361c1e5178b69a1531b238c8460} | 0 ...join_8-6-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...join_8-7-d964114ed76536c8e3cacd231340851c} | 0 ...join_8-8-e44aff8a860cf3965752d3e1ce725cde} | 0 ...join_8-9-ae582a255a02d91674aab500aee79e20} | 0 ..._join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._join_9-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...join_9-10-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_9-11-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_9-13-94538bc7322522a5534cafc0551d2189 | 2 +- ...join_9-15-63261d35ddda973eeeb97b994ab7a476 | 2 +- ...join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 | 2 +- ...join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 | 2 +- ...e_join_9-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_9-21-21269869cd3aaf4ade2170d9017de018 | 2 +- ...join_9-23-4e0e8cd0626a84b21ca7d2f633623578 | 2 +- ...join_9-25-2fe7b834b341bf18e36cd79dd00ec16a | 2 +- ...join_9-27-43ba2c72db9db1ec18d835ec978f8da1 | 2 +- ...join_9-29-b66c416fdd98d76981f19e9c14b6a562 | 2 +- ...join_9-31-b889b147255231f7fe44bd57e1f8ba66 | 2 +- ...join_9-33-aa8d713ad4e19b72b5bd7628d60c295e | 2 +- ...join_9-35-b9d66e78b8898a97a42d1118300fa0ce | 2 +- ...join_9-37-b0ca9e20cd48457e6cf1c313d5505213 | 2 +- ...join_9-38-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_9-40-94538bc7322522a5534cafc0551d2189 | 2 +- ...join_9-42-63261d35ddda973eeeb97b994ab7a476 | 2 +- ...join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 | 2 +- ...join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 | 2 +- ...join_9-48-21269869cd3aaf4ade2170d9017de018 | 2 +- ...join_9-50-4e0e8cd0626a84b21ca7d2f633623578 | 2 +- ...join_9-52-2fe7b834b341bf18e36cd79dd00ec16a | 2 +- ...join_9-54-43ba2c72db9db1ec18d835ec978f8da1 | 2 +- ...join_9-56-b889b147255231f7fe44bd57e1f8ba66 | 2 +- ...join_9-58-aa8d713ad4e19b72b5bd7628d60c295e | 2 +- ...join_9-60-b9d66e78b8898a97a42d1118300fa0ce | 2 +- ...join_9-62-b0ca9e20cd48457e6cf1c313d5505213 | 2 +- ..._join_9-7-24ca942f094b14b92086305cc125e833 | 2 +- ..._join_9-8-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._join_9-9-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._table1-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ..._table2-3-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ...e_union-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ...between-0-df3cf89fcf2ef64199a582fae14a3321 | 1 - ...nstant-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...onstant-0-8c922b2264278dd481ef0dff2088e2b8 | 1 - ...onstant-1-604fde0397444c5c9f1d70e6287de664 | 1 + ..._format-3-84db2ef4a7f2151e26457db559b862d9 | 500 ---- ...olserde-3-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ...olserde-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ...able_1-1-aca7ae366662c9698b5d2c01a6cb3948} | 0 ...table_1-3-d6518380547e7eef338886f3bdc7bdd2 | 2 +- .../cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a | 2 +- ..._to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 | 1 - ...ar_udf-0-4de1b2fa0908a3d856474aae2bc38c08} | 0 ...har_udf-1-5b1e7c580ed5e756d30a4c557af8902} | 0 ...har_udf-2-880ce74a83bb2bfb9c9bd584b8874ac} | 0 ...cluster-1-707a2295731e0d631a6c5f71c745c8d5 | 2 +- ...luster-11-dcf78a6537ba2b4d4b828a9a27cf545e | 2 +- ...luster-13-12635b4b7f34eba5554d5b892b5b64e7 | 2 +- ...luster-15-b4c15c85c18f310f1b5bc56a78ad94b0 | 2 +- ...luster-17-62979aa9e6b4e6ffb44ec452aabbef65 | 2 +- ...luster-19-e5284c7a7c36ee55740bd127ef4bf8c7 | 2 +- ...luster-21-4787b258a786cf195bcb59cd90f6013f | 2 +- ...luster-23-b66ed6ead4deecd49f0f67de1f2bab2e | 2 +- ...luster-25-f57ce48b6a6e671b58c96535ab482b6a | 2 +- ...cluster-3-530671e2a5b8983c60cfedaf486f0f0f | 2 +- ...cluster-5-e99040f6a24c53a4c89801ff3663ff72 | 2 +- ...cluster-7-a22600d60c81a25061b1e20b6726c691 | 2 +- ...cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 | 2 +- ...hortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f | 11 - ..._partlvl-4-30d92b61681b9ae7786ed46c3c3e808 | 58 - ...partlvl-5-f37a302cb19b4fe1c8280c08153294a3 | 129 - ...partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 | 58 - ...partlvl-8-dc5682403f4154cef30860f2b4e37bce | 129 - ..._tbllvl-3-7c45bd1125420b85a0374fecbf947a95 | 73 - ..._tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 | 141 - ..._tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d | 89 - ...ombine1-0-84b74227c9f1563f530cd3ac3b333e54 | 2 +- ...ombine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 2 +- ...ombine1-2-c95dc367df88c9e5cf77157f29ba2daf | 2 +- ...ombine1-3-6e53a3ac93113f20db3a12f1dcf30e86 | 2 +- ...ombine1-4-84967075baa3e56fff2a23f8ab9ba076 | 2 +- ...ombine1-5-2ee5d706fe3a3bcc38b795f6e94970ea | 2 +- ...ombine1-6-1d1f97cce07323812de3027920b04b75 | 2 +- ...ombine1-9-e5ce23369b0ad260512a0f61c6969b73 | 2 +- ...adoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...doop20-12-cd15ffd140539cf86090814729ec4748 | 8 - ...doop20-14-4695309eb4e91ef29c9857aa8fd6130c | 12 - ...adoop20-16-557997716a68312e8cae75428e3ce31 | 1 - ...doop20-18-2af7419c1d84fe155e23f3972e049b97 | 2 - ...adoop20-2-c95dc367df88c9e5cf77157f29ba2daf | 1 - ...adoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 | 1 - ...adoop20-4-84967075baa3e56fff2a23f8ab9ba076 | 1 - ...adoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea | 1 - ...adoop20-6-a4fb8359a2179ec70777aad6366071b7 | 1 - ...adoop20-7-16367c381d4b189b3640c92511244bfe | 1 - ...adoop20-8-99d1f07b2ce904afd6a809fd1814efe9 | 1 - ...adoop20-9-30cb07965e4b5025545361b948fc83c2 | 1 - ...ne2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...ne2_win-1-c95dc367df88c9e5cf77157f29ba2daf | 1 - ...e2_win-11-cd15ffd140539cf86090814729ec4748 | 8 - ...e2_win-13-4695309eb4e91ef29c9857aa8fd6130c | 12 - ...ne2_win-15-557997716a68312e8cae75428e3ce31 | 1 - ...e2_win-17-2af7419c1d84fe155e23f3972e049b97 | 2 - ...ne2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 | 1 - ...ne2_win-3-84967075baa3e56fff2a23f8ab9ba076 | 1 - ...ne2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea | 1 - ...ne2_win-5-a4fb8359a2179ec70777aad6366071b7 | 1 - ...ne2_win-6-16367c381d4b189b3640c92511244bfe | 1 - ...ne2_win-7-99d1f07b2ce904afd6a809fd1814efe9 | 1 - ...ne2_win-8-30cb07965e4b5025545361b948fc83c2 | 1 - ...ombine3-11-30b8b79049113252bec1cbeac4018a3 | 12 - ...mbine3-12-f4baee25e0ad813258d67d707a6fc43b | 12 - ...mbine3-13-1c359bedf474e8e26f3b3562f7af6edc | 30 - ...mbine3-21-8ba8f8723c1530be062cefc2d9246e56 | 30 - ...binary-1-44b15c6c9600109e064a5ea5f9c81051} | 0 ...oolean-1-72ee4bdd5cea69136940dc40e6890e1d} | 0 ...boolean-2-fbea367698de65e22d4d660a518ea95e | 2 +- ...boolean-3-a14d8a5835c94829271f9f463d96d83d | 2 +- ...double-1-8f634b9e334fd58e71844e6283d9794d} | 0 ...s_long-1-9313f166464633b3929707d7ef11d758} | 0 ...string-1-3491ef2747a8f34899108d4ae8ebc7eb} | 0 ..._string-1-db089ff46f9826c7883198adacdfad59 | 4 +- ...izer1-26-5522db58d123d1bec48b6e71c1b258f3} | 0 ...izer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65} | 0 ...mizer1-33-7722bcc896718b584f78cecdab1fdc9f | 16 + ...izer1-35-e1d4857548e626bb31d70c096b8d0a95} | 0 ...mizer1-36-7722bcc896718b584f78cecdab1fdc9f | 16 + ...izer1-38-ef6502d6b282c8a6d228bba395b24724} | 0 ...izer1-39-ea87e76dba02a46cb958148333e397b7} | 0 ...izer1-41-b79b220859c09354e23b533c105ccbab} | 0 ...izer1-42-ea87e76dba02a46cb958148333e397b7} | 0 ...izer1-44-638e5300f4c892c2bf27bd91a8f81b64} | 0 ...izer1-45-66010469a9cdb66851da9a727ef9fdad} | 0 ...imizer1-47-3514c74c7f68f2d70cc6d51ac46c20} | 0 ...izer1-48-66010469a9cdb66851da9a727ef9fdad} | 0 ...izer1-49-b9d963d24994c47c3776dda6f7d3881f} | 0 ...izer1-50-7490df6719cd7e47aa08dbcbc3266a92} | 0 ...izer1-51-e71195e7d9f557e2abc7f03462d22dba} | 0 ...izer1-52-777edd9d575f3480ca6cebe4be57b1f6} | 0 ...izer1-53-73da9fe2b0c2ee26c021ec3f2fa27272} | 0 ...izer1-54-e71195e7d9f557e2abc7f03462d22dba} | 0 ...izer1-55-b1e2ade89ae898650f0be4f796d8947b} | 0 ...izer1-57-fcf9bcb522f542637ccdea863b408448} | 0 ...mizer1-58-3070366869308907e54797927805603} | 0 ...izer1-60-dad56e1f06c808b29e5dc8fb0c49efb2} | 0 ...mizer1-61-3070366869308907e54797927805603} | 0 ...izer1-62-b9d963d24994c47c3776dda6f7d3881f} | 0 ...izer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4} | 0 ...mizer1-64-a6bba6d9b422adb386b35c62cecb548} | 0 ...izer1-65-777edd9d575f3480ca6cebe4be57b1f6} | 0 ...izer1-66-d6bbaf0d40010159095e4cac025c50c5} | 0 ...mizer1-67-a6bba6d9b422adb386b35c62cecb548} | 0 ...mizer15-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer15-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer15-3-e149747103059314a9984235702b24b6 | 2 +- ...mizer15-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer15-5-9914f44ecb6ae7587b62e5349ff60d04 | 2 +- ...mizer15-7-e149747103059314a9984235702b24b6 | 2 +- ...imizer2-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer2-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-10-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer2-12-e6b368bfaea4d2838f8038b3bd29db06 | 2 +- ...mizer2-13-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-15-18f10d12e8bfa473a916c2f528500538 | 2 +- ...mizer2-16-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer2-18-18f10d12e8bfa473a916c2f528500538 | 2 +- ...mizer2-19-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e | 2 +- ...mizer2-22-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e | 2 +- ...mizer2-25-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-27-9b22dad2843cdc379d90687745561104 | 2 +- ...mizer2-28-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer2-3-d915fbdd493869aec42f548bdb66598d | 2 +- ...mizer2-30-9b22dad2843cdc379d90687745561104 | 2 +- ...mizer2-31-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b | 2 +- ...mizer2-34-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b | 2 +- ...imizer2-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer2-6-d915fbdd493869aec42f548bdb66598d | 2 +- ...imizer2-7-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...imizer2-9-e6b368bfaea4d2838f8038b3bd29db06 | 2 +- ...imizer3-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer3-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer3-10-e3d5ff08760b877d49c0f10f63776325 | 2 +- ...mizer3-11-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer3-12-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer3-14-1f13e03988991067d13a9f3f1b36fcf5 | 2 +- ...mizer3-15-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer3-17-1f13e03988991067d13a9f3f1b36fcf5 | 2 +- ...mizer3-18-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer3-19-24ca942f094b14b92086305cc125e833 | 2 +- ...mizer3-21-1f13e03988991067d13a9f3f1b36fcf5 | 2 +- ...imizer3-3-e3d5ff08760b877d49c0f10f63776325 | 2 +- ...imizer3-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer3-6-e3d5ff08760b877d49c0f10f63776325 | 2 +- ...imizer3-7-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer3-8-24ca942f094b14b92086305cc125e833 | 2 +- ...mizer4-1-c0f14def6a135cc50cba364e810ce28e} | 0 ...mizer4-10-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e | 2 +- ...mizer4-13-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-14-24ca942f094b14b92086305cc125e833 | 2 +- ...mizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e | 2 +- ...mizer4-17-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer4-18-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer4-20-6ad79a473a876923a247f0cb57bb4208 | 2 +- ...mizer4-21-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-23-6ad79a473a876923a247f0cb57bb4208 | 2 +- ...mizer4-24-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-26-62a0fd05be48759c39f3c284458dde9b | 2 +- ...mizer4-27-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf | 2 +- ...mizer4-3-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...mizer4-30-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf | 2 +- ...mizer4-33-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 | 2 +- ...mizer4-36-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer4-38-85fff71b240d0e26bab91d670c1349f2 | 2 +- ...mizer4-39-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-41-85fff71b240d0e26bab91d670c1349f2 | 2 +- ...mizer4-5-b5777cff7c522c4b527f77988e7f6bf1} | 0 ...imizer4-6-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer4-7-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...imizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e | 2 +- ...imizer6-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer6-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-10-25b1fe48ef86952fc7766fb74b63bf21 | 2 +- ...mizer6-11-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer6-12-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-14-84463190baec77d61b287a071c8886db | 2 +- ...mizer6-15-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-17-84463190baec77d61b287a071c8886db | 2 +- ...mizer6-18-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de | 2 +- ...mizer6-21-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de | 2 +- ...mizer6-24-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer6-25-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-27-e149747103059314a9984235702b24b6 | 2 +- ...mizer6-28-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer6-3-25b1fe48ef86952fc7766fb74b63bf21 | 2 +- ...mizer6-30-e149747103059314a9984235702b24b6 | 2 +- ...mizer6-31-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-33-15d991127dc684513e2fff1aea3f1560 | 2 +- ...mizer6-34-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-36-15d991127dc684513e2fff1aea3f1560 | 2 +- ...mizer6-37-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-39-813d72763b5e9e3bed0f06232f55c8b8 | 2 +- ...imizer6-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-40-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-42-813d72763b5e9e3bed0f06232f55c8b8 | 2 +- ...mizer6-43-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-45-ff6c54b483726ef15c90a4c68dc659a0 | 2 +- ...mizer6-46-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-48-ff6c54b483726ef15c90a4c68dc659a0 | 2 +- ...mizer6-49-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-51-4746d944f4193018017984ca2df3c60d | 2 +- ...mizer6-52-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-54-4746d944f4193018017984ca2df3c60d | 2 +- ...mizer6-55-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-56-24ca942f094b14b92086305cc125e833 | 2 +- ...mizer6-58-4746d944f4193018017984ca2df3c60d | 2 +- ...imizer6-6-25b1fe48ef86952fc7766fb74b63bf21 | 2 +- ...imizer6-7-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer6-8-24ca942f094b14b92086305cc125e833 | 2 +- ...imizer7-0-24ca942f094b14b92086305cc125e833 | 2 +- ...imizer7-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer7-10-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer7-12-22d71fb589c53776dabb4696b38c4a42 | 2 +- ...mizer7-13-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer7-15-22d71fb589c53776dabb4696b38c4a42 | 2 +- ...imizer7-3-22d71fb589c53776dabb4696b38c4a42 | 2 +- ...imizer7-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer7-6-22d71fb589c53776dabb4696b38c4a42 | 2 +- ...imizer7-7-24ca942f094b14b92086305cc125e833 | 2 +- ...imizer7-8-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...imizer7-9-4d3e60a0e9bd8c12fdba4e010493537d | 2 +- ...imizer8-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer8-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer8-10-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...timizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f | 2 +- ...mizer8-13-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer8-15-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer8-3-932db4b9935e4bc3d21dd33a8d12c275 | 2 +- ...imizer8-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer8-6-932db4b9935e4bc3d21dd33a8d12c275 | 2 +- ...imizer8-7-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...ptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f | 2 +- ... count-1-c47c4abedf055b4e734232fd2e274d55} | 0 .../count-10-d0f804c7d06375db74a0fcf5f17603c6 | 2 +- .../count-12-944f53db544c07a7b38a0544a21d8e13 | 2 +- .../count-2-461bad3feb7dbc25fb35d45c6876d698 | 2 +- .../count-3-dbcec232623048c7748b708123e18bf0 | 2 +- .../count-5-d0f804c7d06375db74a0fcf5f17603c6 | 2 +- .../count-7-944f53db544c07a7b38a0544a21d8e13 | 2 +- .../count-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...db name-1-417609d2bb67ba26de38e92ad834008f | 500 ---- ...able as-1-b9002c1d71895be765575b62656d1928 | 2 +- ...reate_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d | 1 - ...eate_1-10-b9c0b95624e601614ea2561b83aaf0ba | 4 - ...eate_1-12-fe8680bc5ef6fe3e64c68d2638e10785 | 2 - ...reate_1-13-5715f85456733a71fb7c83f1b254b3a | 4 - ...eate_1-15-fd9415e340cf54a9473cc4bf86673816 | 2 - ...eate_1-16-c99c700ca511b68577aae8ae513a4a32 | 4 - ...reate_1-2-ecd02bc3563cd6b60b8394956cb69084 | 2 - ...reate_1-3-c27702ff131e0ecfd71f1e1779fbe365 | 4 - ...reate_1-6-52dc9f900d7f7a559698aff9565f061a | 2 - ...reate_1-7-8564aa6cc2b0ee85292b3e8976fe9001 | 4 - ...reate_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 | 2 - ..._escape-1-ecd02bc3563cd6b60b8394956cb69084 | 2 - ..._escape-2-c27702ff131e0ecfd71f1e1779fbe365 | 4 - ...e_escape-4-7d84873a6ce03e0e408fa67ef5dd733 | 1 - ...ricudaf-0-499d36ed8229cbf74a07b59914bdf717 | 9 - ...ricudaf-3-d3b1af3e7f037de800255d9411a690e8 | 1 - ...ericudf-0-dd23fb149bb6d6937b838334559ad8d1 | 10 - ...tformat-2-8c2e4081b9d69297c35faab0a2ff86dc | 4 +- ...utformat-5-8552731917a8260c25e6df79b83bf5c | 4 +- ...tformat-8-33a4312fd617c5c9f2a560dc6ce868a5 | 4 +- ...ate_like-1-30485a2507b60b96cad3d293527e6af | 27 - ...te_like-10-7d84873a6ce03e0e408fa67ef5dd733 | 1 - ...e_like-11-ba64f0122b21f605d8b2928753784130 | 2 - ...te_like-3-eea111a209cf8895f31f64699669c705 | 27 - ...te_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 | 28 - ...e_view-10-eea111a209cf8895f31f64699669c705 | 10 +- ...e_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 | 4 +- ...e_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 | 10 +- ...e_view-25-87a663f8fd80110a9cee249535037c0d | 14 +- ...ke_view-5-ecd02bc3563cd6b60b8394956cb69084 | 4 +- ...ike_view-6-30485a2507b60b96cad3d293527e6af | 10 +- ...ke_view-9-52dc9f900d7f7a559698aff9565f061a | 4 +- ...ressed-10-d6fee0f05fa9b04cb7c557862402c929 | 14 - ...ressed-11-614c34f9e88015f21bffc4b8930bc95d | 1 - ...ressed-12-4d89cbe49f710527b54e6262472f0320 | 1 - ...pressed-2-84b74227c9f1563f530cd3ac3b333e54 | 1 - ...pressed-6-d6fee0f05fa9b04cb7c557862402c929 | 14 - ...pressed-7-614c34f9e88015f21bffc4b8930bc95d | 1 - ...pressed-8-4d89cbe49f710527b54e6262472f0320 | 1 - ...ed_type-1-ecd02bc3563cd6b60b8394956cb69084 | 8 +- ...ed_type-2-c27702ff131e0ecfd71f1e1779fbe365 | 10 +- ...d_type-3-280fd0d9876e475d7dcf5383876ebc79} | 0 ...ce_view-2-30dc3e80e3873af5115e4f5e39078a13 | 30 - ..._table1-0-cafed8ca348b243372b9114910be1557 | 2 +- ..._table1-4-f3f1642674545762a4bff5cb75634e20 | 12 +- ..._table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e | 12 +- ..._table1-6-d7a147c6b0a3609663628b43457b2cef | 14 +- ..._table-1-719851d0e8b89b51bdc6be4177455a92} | 0 ...te_udaf-0-a69c2b11dc912ef5444af32dce6aa33e | 10 - ...e_view-19-df2da604b5a8f2b236519510b5e4d34b | 1 - ...itioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 | 13 - ...itioned-5-d7a7d8592fca266745725192d3f875fc | 36 - ...heck_1-0-d782db598869f9b19e0fcf5ea2a83594} | 0 ...heck_1-1-bd8395c55fa2fc80b68eb043b7020b76} | 0 ...heck_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617} | 0 ...heck_1-3-32a3e6eb858d37f58e225d07fb323254} | 0 ...heck_1-4-36a6b6fb71ea08ff817dd40d1ffdb970} | 0 ...heck_1-5-103739f072b849d212dbc40919f92b74} | 0 ...heck_1-6-9a202e3bf15be2a310504a50920e7d25} | 0 ...heck_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac} | 0 ...heck_2-0-d782db598869f9b19e0fcf5ea2a83594} | 0 ...heck_2-1-bd8395c55fa2fc80b68eb043b7020b76} | 0 ...heck_2-2-24ca942f094b14b92086305cc125e833} | 0 ...heck_2-3-44d382ce6848d3f0b900b0808747d8e9} | 0 ...heck_2-4-c14792ccac2ca64e3e9e21af4fd12d2c} | 0 ...heck_2-5-32a3e6eb858d37f58e225d07fb323254} | 0 ...heck_2-6-36a6b6fb71ea08ff817dd40d1ffdb970} | 0 ...heck_2-7-103739f072b849d212dbc40919f92b74} | 0 ...heck_2-8-9a202e3bf15be2a310504a50920e7d25} | 0 ...heck_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac} | 0 ...varchar-7-8620d7f55849ab163b6b85f51abdaaec | 5 - ...varchar-8-c93df93c0e8688f9c7a6167589670d32 | 5 - ...le.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd | 2 +- ...e.table-0-c657beb729b6a7882309a203fc6f298e | 2 +- ..._drop-10-8db536f925bf0f5058f97897e145a661} | 0 ..._drop-11-1b0a6cff3151cfa0ef0a6f78587973a5} | 0 ..._drop-12-2ea883422b74b701711e14e61472ba06} | 0 ..._drop-13-e02a53f7e798d2741152526516f14941} | 0 ..._drop-14-8f0fe60664d020b2a42c7f5c0c7bed35} | 0 ..._drop-15-7928ac876f76c990fa21d74b6c9e14f6} | 0 ..._drop-16-f9036ff3f0a3101373bdbc9a52faf00e} | 0 ..._drop-17-95b906fd73935da8746b5277170e91e8} | 0 ..._drop-18-f65bf675b83871af7906741a60fa1318} | 0 ..._drop-19-d7fefbf585dbb67491e871ef58dca752} | 0 ..._drop-20-cbb84f0db4d55930a85cff28f7400bd0} | 0 ..._drop-21-eea49f066631be60519ae9d6b614d7d0} | 0 ..._drop-22-85833c3a68c29098827e438ff580bb94} | 0 ..._drop-23-84a5672989118a1b5792474c1469de90} | 0 ..._drop-24-441116797e8d95554c74472fa7644440} | 0 ..._drop-25-847ca97dd211889d83e5d487bfc44e23} | 0 ..._drop-26-bea20178515df24fcca04c0384f1c1b7} | 0 ..._drop-27-cb4b90a7f63c1646319ee7bb014a7750} | 0 ..._drop-28-53d67cbed634cff012dac41340bf6630} | 0 ..._drop-29-29d3232325eda40cbb0bd1786b7d070e} | 0 ...e_drop-3-db64b724719d27c7f0db4f51f5c4edaa} | 0 ..._drop-30-bbad0860316d8b9b1eed50d231f3ab5d} | 0 ..._drop-31-981f8e58caeae9cbbad3a113e3043de5} | 0 ..._drop-32-6c8e7590238b5aca1772721f0b914ece} | 0 ..._drop-33-2bc7864932f597bdf98bdc410b1a2d9c} | 0 ...e_drop-34-491138bed44a70cb783bb2b531e1d82} | 0 ..._drop-35-9e0285d0596607cdadf75a763a543866} | 0 ..._drop-36-e66471f3f1bbe2d4b3b214e47793656d} | 0 ..._drop-37-f6410721e3125a89836817136306eac4} | 0 ..._drop-38-7d45d97adebe50917a94cbe232c112a8} | 0 ..._drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1} | 0 ...e_drop-4-f21dd011aeb417043ed58c03fd5c3bf0} | 0 ..._drop-40-10073fb74a5c792322fc52389997695b} | 0 ..._drop-41-7164c585e1ef4d9036ed4db275811084} | 0 ..._drop-42-c55cffbfc4d950363be97ccdb028faf3} | 0 ..._drop-43-cc6860c125b8b62450cb858c72716dc2} | 0 ..._drop-44-de81fd80132350aedcd9f0d9a212fd94} | 0 ..._drop-45-64e22634663e57153119340583e93651} | 0 ..._drop-46-eb28b907b605e51026f9902287e1d90d} | 0 ..._drop-47-44e4d2bdab2dfa4583da47281ed00ba3} | 0 ..._drop-48-d419f4ff197d4291208c2028cd158909} | 0 ...e_drop-5-d419f4ff197d4291208c2028cd158909} | 0 ...e_drop-6-b7cf74929eabe781b0db79ed1043dc24} | 0 ...e_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9} | 0 ...e_drop-8-b3980119a4b2d8f531951a940cba3697} | 0 ...e_drop-9-97101266791d2b2c662bcde549422318} | 0 ...location-1-2b66b4554cf8ecf2ab6c325d4d89491 | 2 +- ...cation-10-c5cd9c57a13da7f345563fbd75da4e45 | 12 +- ...cation-11-9c36cac1372650b703400c60dd29042c | 2 +- ...ocation-4-be65cf649363681d54e593c42a5ecffb | 12 +- ...ocation-5-9c36cac1372650b703400c60dd29042c | 2 +- ...ocation-7-5698ac10441da07dbe3a947143c999c2 | 2 +- ...perties-0-e7bfbd9422685e9a3a6c9bd4965f828f | 2 +- ...perties-1-10de6a198e2b3f61974519ddd8623e68 | 2 +- ...erties-10-26c10ff2ec4a69b16589069ced427d23 | 2 +- ...perties-3-751417d45b8e80ee5cba2034458b5bc9 | 2 +- ...perties-5-51c0974df1125b233936f25ce709ba4a | 2 +- ...perties-6-26c10ff2ec4a69b16589069ced427d23 | 2 +- ...perties-7-e7bfbd9422685e9a3a6c9bd4965f828f | 2 +- ...perties-8-10de6a198e2b3f61974519ddd8623e68 | 2 +- ...date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1} | 0 ...date_3-3-c26f0641e7cec1093273b258e6bf7120} | 0 ...date_4-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...date_4-1-b84f7e931d710dcbe3c5126d998285a8} | 0 ...date_4-2-6272f5e518f6a20bc96a5870ff315c4f} | 0 ...date_4-3-4a0e7bde447ef616b98e0f55d2886de0} | 0 ...date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8} | 0 .../date_4-5-bee09a7384666043621f68297cee2e68 | 1 + ...date_4-6-b84f7e931d710dcbe3c5126d998285a8} | 0 ...arison-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...arison-1-69eec445bd045c9dc899fafa348d8495} | 0 ...rison-10-bcd987341fc1c38047a27d29dac6ae7c} | 0 ...rison-11-a9f2560c273163e11306d4f1dd1d9d54} | 0 ...rison-12-4a7bac9ddcf40db6329faaec8e426543} | 0 ...arison-2-fcc400871a502009c8680509e3869ec1} | 0 ...arison-3-b8598a4d0c948c2ddcf3eeef0abf2264} | 0 ...arison-4-14d35f266be9cceb11a2ae09ec8b3835} | 0 ...arison-5-c8865b14d53f2c2496fb69ee8191bf37} | 0 ...arison-6-f2c907e64da8166a731ddc0ed19bad6c} | 0 ...arison-7-5606505a92bad10023ad9a3ef77eacc9} | 0 ...parison-8-47913d4aaf0d468ab3764cc3bfd68eb} | 0 ...arison-9-1e5ce4f833b6fba45618437c8fb7643c} | 0 ..._join1-2-e967e1ef6b209dfa5bdc60021dcb1964} | 0 ..._serde-7-580096b3b48db26bea91b80e1e1b081a} | 0 ...te_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d} | 0 ...ddltime-1-426da52526f6f48c0ddeb0026fd566f1 | 4 - ...cimal_1-10-be179d261b9c42ed843dbf736b12e75 | 1 - ...imal_1-11-4c1fefa61e10a78f6406b526a60e4937 | 1 - ...imal_1-12-75b9add5e32a963cc9913c6ef4f84989 | 1 - ...cimal_1-5-cbe6b235663cf78e602673ed715a2f40 | 1 - ...cimal_1-6-91b7219bd8c67673e16cb970fcc7be1b | 1 - ...decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a | 1 - ...cimal_1-8-cdd0932288d3cc43636334439805769d | 1 - ...cimal_1-9-1504e1b00092e2c58bafcc9936ad178c | 1 - ...imal_2-10-f97d72aeb605ee18d34361c073552e92 | 1 - ...imal_2-11-58a090c30c59446d1e2b2a6c85fabf50 | 1 - ...imal_2-14-3105d1029ad981af9cf1039ad9410fc0 | 1 - ...imal_2-15-3266fde6f6ab80a8bc027de0d5574f02 | 1 - ...imal_2-16-dc565c6c5bc24fd8b32729ce91999580 | 1 - ...imal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d | 1 - ...imal_2-18-f7c34d67fd579c82c636415172ec675e | 1 - ...imal_2-19-f97d72aeb605ee18d34361c073552e92 | 1 - ...imal_2-20-58a090c30c59446d1e2b2a6c85fabf50 | 1 - ...imal_2-21-d72d68be1217c7b7a958f58456d85821 | 1 - ...imal_2-22-648e694eea042c59e8db30d067cb5bc8 | 1 - ...imal_2-23-a87b701c93a922b9e33ba16ae6a477ff | 1 - ...imal_2-27-a5ea3949eb5ab338916e4316c676c7f6 | 1 - ...imal_2-28-4a5410f96c6ef0843f12b0f593c104b1 | 1 - ...imal_2-30-26a71d79e41353830b4ada96de6e2b8a | 1 - ...cimal_2-5-3105d1029ad981af9cf1039ad9410fc0 | 1 - ...cimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 | 1 - ...cimal_2-7-dc565c6c5bc24fd8b32729ce91999580 | 1 - ...cimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d | 1 - ...cimal_2-9-f7c34d67fd579c82c636415172ec675e | 1 - ...imal_3-10-420614bb0789115e008c96a7ad822624 | 4 - ...imal_3-11-63913753553b16d6c24e063fb49fdd15 | 4 - ...cimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b | 38 - ...cimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 | 38 - ...cimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b | 38 - ...cimal_3-6-127a3a8400cae591c282dd24f8951e55 | 30 - ...cimal_3-7-9d4f27d4a4819113c5083462baa72052 | 30 - ...cimal_3-8-f65f4df6e3d971d575654ade4b4e4800 | 17 - ...cimal_3-9-b54243d38214362f9a9b1831548faac4 | 56 - ...imal_4-2-85c3185beb011f5c1e6856fc773a7484} | 0 ...imal_4-3-1451d7491441c1632fd5f751876cce6e} | 0 ...imal_4-4-1bf9ff1d72a06c33885ba695adf2511d} | 0 ...cimal_4-6-693c2e345731f9b2b547c3b75218458e | 2 +- ...cimal_4-7-f1eb45492510cb76cf6b452121af8531 | 2 +- ...l_join-0-4668e9dee2cd7a32f2b7311d7cd35508} | 0 ...l_join-1-5098974222b22a21ed847c7906df9313} | 0 ...ecision-3-42cb35d680b3caeeb22e1c4865b8264b | 75 - ...ecision-4-38aaeba3e587b4dac72e26c4b02029fc | 75 - ...ecision-5-bb27734245ecbd0511be91af21c3b9ef | 75 - ...recision-6-b2547e6ef33325b2da12ce91b57af21 | 75 - ...ecision-7-ee5b7767c7fbb8614bb4ef907e8737b7 | 75 - ...ecision-8-6e6bd4655de124dc1fc482ce0d11930e | 75 - ...ecision-9-e7b465fbeb49487b2a972a314e2c01ab | 1 - ...on_name-1-9de8e5f66c536d4ace89c61759db829c | 2 +- ...on_name-3-a7047012b4bce0158edaafe5cf0a57be | 2 +- ...imiter-1-121ff21e6931a82235de8301118cbed8} | 0 ...limiter-2-d7137294d2e53ea6edc259943e4c6069 | 2 +- ...limiter-3-176724f76343433a8f2e6131b12206d7 | 2 +- ..._indent-1-5536eb772d43014b971c6da3a0c44904 | 6 - ..._indent-2-91bffa09f4f2caaaa6864bf935c2ea94 | 32 - ...se_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 1 - ...e_json-11-b05391400dc31139998dc3abaaf86320 | 1 - ...ase_json-2-8e7cfe3e6069e796124ca940125385a | 1 - ...se_json-3-d097973152d91fa8072facb0f739e304 | 1 - ...se_json-4-549981e00a3d95f03dd5a9ef6044aa20 | 1 - ...se_json-5-a3ee372283f45479db3f2cd7ebeedc8c | 1 - ...ase_json-8-8e7cfe3e6069e796124ca940125385a | 1 - ...se_json-9-d097973152d91fa8072facb0f739e304 | 1 - ...itioned-3-b80c7ae3530bfdbc8e865d49742da826 | 8 +- ...ed_json-4-b80c7ae3530bfdbc8e865d49742da826 | 1 - ..._pretty-1-dbfaa12f26f99277b8397379189172cf | 1 - ...pretty-10-dbfaa12f26f99277b8397379189172cf | 1 - ...pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 | 12 - ...pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 | 1 - ...pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 | 24 - ...pretty-15-db4abe06aba81e685c52db6b43ba7c03 | 1 - ...pretty-16-ada55b65b72e0d65563ad6161e005f22 | 37 - ..._pretty-2-713712e0f6f18144d1f3a522e9b98861 | 23 - ..._pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 | 1 - ..._pretty-4-1546db18568697fa68a7fc781802d255 | 14 - ..._pretty-5-ce1966d8626096821b404ab8745c4914 | 1 - ..._pretty-6-1546db18568697fa68a7fc781802d255 | 44 - ..._pretty-7-db4abe06aba81e685c52db6b43ba7c03 | 1 - ..._pretty-8-1546db18568697fa68a7fc781802d255 | 31 - ...syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 | 12 - ...syntax-11-ab161e38c5d66b6c344c8372160ac74f | 33 - ...syntax-12-90c7890e1aa28e94520f35f5679560a4 | 1 - ...syntax-13-7c1216f9852d454bf93256e5a2588758 | 1 - ...syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 | 3 - ...syntax-15-e420b255509153b3326f00dcd25d11e4 | 1 - ...syntax-16-5043ee273a313d28adeca74fd33739a7 | 1 - ...syntax-17-c97a9e691cc08199678ead7f79d58b58 | 3 - ...syntax-18-69cecdc1889b7feb40b46c95524aaad4 | 1 - ...syntax-19-c1c3359705e256d7641bbffab00c43fa | 1 - ...syntax-20-781dd449a9b26dc54f3bba8e5178cd8a | 3 - ...syntax-21-5bac87eeb7e71928d01275b006720de3 | 10 - ...syntax-22-719a15ffd0018bb2898f9045be886e0f | 12 - ...syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 | 32 - ...syntax-24-ee226b42db35b219702319858e925468 | 10 - ...syntax-25-b6e10514fb473803c44bc793b9f9713e | 12 - ...syntax-26-4851dc65e26ec33c605ab1ed65b59bec | 32 - ..._syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 | 10 - ..._syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d | 12 - ...be_syntax-8-9e40f8077f34fa35fccfeae972e8c4 | 33 - ..._syntax-9-fb744775fb30d92297534d29b6eafd6b | 10 - ...le_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 1 - ...le_json-2-9c36cac1372650b703400c60dd29042c | 1 - ...le_json-3-576670af142149302decb9bf8662e68a | 1 - ...le_json-4-4a57591b392bb8fd18238d068d191721 | 1 - ...le_json-6-ac49d26a0211b804fee89bbe0808f430 | 1 - ...le_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 | 1 - ...le_json-9-b05391400dc31139998dc3abaaf86320 | 1 - ..._formats-4-a4890f2b20715c75e05c674d9155a5b | 2 +- ...t_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c | 2 +- ..._check-2-2d27f92dfced693fa3a68ecce5e2e838} | 0 ..._check-4-a2150709a6ff73326bdf4865dd124a23} | 0 ...ge_off-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...nge_off-0-a071dedef216e84d1cb2f0de6d34fd1a | 1 - ...ge_off-1-a071dedef216e84d1cb2f0de6d34fd1a} | 0 ...nge_off-1-ce3797dc14a603cba2a5e58c8612de5b | 1 - ...ge_off-2-ce3797dc14a603cba2a5e58c8612de5b} | 0 ...ge_off-3-f5340880d2be7b0643eb995673e89d11} | 0 ...nge_off-4-34064fd15c28dba55865cb8f3c5ba68c | 1 - ...ge_off-4-714ab8c97f4d8993680b91e1ed8f3782} | 0 ...nge_off-5-34064fd15c28dba55865cb8f3c5ba68c | 1 + ...ge_off-6-f40a07d7654573e1a8517770eb8529e7} | 0 ..._stats-0-418ec894d08c33fd712eb358f579b7a0} | 0 ..._stats-1-10987e425ba8ba8d9c01538f16eab970} | 0 ..._stats-2-a2d8f812612283b20ec3f1e92a263440} | 0 ...t_stats-3-a4397664f1f109ef0aa5ff36961b25b} | 0 ..._stats-4-e540680af4a857404a0cb7cabc1bcf31} | 0 ..._stats-5-32e9736bf27c1d2e4399a8125e14befc} | 0 ..._stats-6-297a53801744e6c4786e315e32c6189a} | 0 ..._stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d} | 0 ..._stats-8-ca16024e6f5399b1d035f5b9fd665163} | 0 ..._stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8} | 0 ...n_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee} | 0 ...n_dirs-3-46fe5bb027667f528d7179b239e3427f} | 0 ...n_dirs-4-26dcd2b2f263b5b417430efcf354663a} | 0 ...n_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea} | 0 ...n_dirs-6-7a9e67189d3d4151f23b12c22bde06b5} | 0 ...n_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd} | 0 ...n_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea} | 0 ...n_dirs-9-40110efef10f6f7b873dcd1d53463101} | 0 ...unction-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 2 +- ...p_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 2 +- ...n_dirs-0-a99af48bbcbaba062e9bc387ae2b4975} | 0 ...n_dirs-1-e67a0f7ff61a97b2c49386890ea88c54} | 0 ...n_dirs-2-19915227905aab376d918b3cada85c25} | 0 ...n_dirs-3-e32b952789a115ff02201dfa618d92b2} | 0 ...n_dirs-4-58aff7fa05fba3a7549629a17e285036} | 0 ...n_dirs-5-e32b952789a115ff02201dfa618d92b2} | 0 ...n_dirs-6-4642f8a18bf7409849f2e91d7a05f352} | 0 ...itions-10-9120e865aa132bac6e0a29c907f0b760 | 2 +- ...titions-4-9120e865aa132bac6e0a29c907f0b760 | 2 +- ...titions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 | 12 - ...titions-7-9120e865aa132bac6e0a29c907f0b760 | 2 +- ...titions-8-46a4f646bbc04f70e7ae5ed992f102f1 | 2 +- ..._filter-1-e3eda6672f6602a1b9bc93ef789662f6 | 12 +- ...filter-11-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-13-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-15-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-17-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-19-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-21-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-22-46a4f646bbc04f70e7ae5ed992f102f1 | 2 +- ...filter-24-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter2-1-e3eda6672f6602a1b9bc93ef789662f6 | 12 +- ...ilter2-10-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...ilter2-12-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...lter2-13-8117981303487dc4c4873356931ef26a} | 0 ...ilter2-14-83e3e422cdf9403523fa60d75376d7d7 | 1 + ...lter2-15-67d75c72ea2d3982c55f3a850d93f83c} | 0 ...ilter2-7-f34625fd49a5e655cba3abb5cb8c5417} | 0 ...filter2-8-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter3-1-e3eda6672f6602a1b9bc93ef789662f6 | 12 +- ...ilter3-10-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...ilter3-12-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter3-8-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...otection-3-312aa26fdea6da7907e3a91f75e36f1 | 10 +- ...p_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 2 +- ..._table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 | 2 +- ..._table2-5-1fdd850f6c301619f91eb58c890f2ad4 | 2 +- ...n_dirs-0-97b52abf021c81b8364041c1a0bbccf3} | 0 ...n_dirs-1-f11a45c42752d06821ccd26d948d51ff} | 0 ...n_dirs-2-c0b85445b616f93c5e6d090fa35072e7} | 0 ...n_dirs-3-10a71bca930d911cc4c2022575b17299} | 0 ...n_dirs-4-b2ca31dd6cc5c32e33df700786f5b208} | 0 ...n_dirs-5-10a71bca930d911cc4c2022575b17299} | 0 ...n_dirs-6-d1c175a9d042ecd389f2f93fc867591d} | 0 ...efault-0-3cd14bc5b126ff8b337c4abc09134260} | 0 ...default-1-16367c381d4b189b3640c92511244bfe | 2 +- ...efault-2-ce929ee6a92b81d8080ca322c1c38a4b} | 0 ...efault-3-f3a5a998099b756a21cf9122a15b09d5} | 0 ...efault-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49} | 0 ...efault-5-fed732d2e7d94a4fc02e7694f9f9a39c} | 0 ...efault-6-adab6a0187003ab7ee6f217c9e409d91} | 0 ...default-7-e707e693aa61edf87768fb71f6e936e1 | 2 +- ...efault-8-adab6a0187003ab7ee6f217c9e409d91} | 0 ...efault-9-4b099f87e221b8fd5c0d0d4a97c0d146} | 0 ...e_order-2-ffe97dc8c1df3195982e38263fbe8717 | 1 - ...e_order-7-5fb418612e7c2201790d6f318c715ccf | 10 - ...e_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 | 10 - ...escape1-6-134b596abc363f0bfa7f770732ebb960 | 1 - ...escape1-7-486585cbb4de5bc908dde4c601dd7c17 | Bin 252 -> 0 bytes ...escape2-8-134b596abc363f0bfa7f770732ebb960 | 1 - ...escape2-9-486585cbb4de5bc908dde4c601dd7c17 | Bin 252 -> 0 bytes ..._compat-5-75eed21390055f8e397c81ab9d253a32 | 1 - ..._compat-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...ll_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ll_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ..._parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ..._parts-11-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...me_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...me_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...ne_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ne_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...overlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...overlap-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ..._rename-5-75eed21390055f8e397c81ab9d253a32 | 1 - ..._rename-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...overlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...overlap-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...managed-3-be31972099603addb71187f19f7cd25d | 1 - ...managed-4-46667daf88f9c8b9b758ced6a6b28ef1 | 1 - ...xternal-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...xternal-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...ocation-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...ocation-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...ocation-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...ocation-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...xisting-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...xisting-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...al_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...al_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...xternal-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...xternal-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...managed-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...xternal-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...xternal-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...ocation-6-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ocation-7-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...ocation-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ocation-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...ocation-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ocation-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...success-6-75eed21390055f8e397c81ab9d253a32 | 1 - ...success-7-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...success-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...success-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...success-5-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...success-6-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...success-5-d2ec90909f243a767be1aa299720f45d | 1 - ...success-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...arrange-0-6f7c8515c354fb050829ebd66413425} | 0 ...rrange-1-378d42317b39c6519f15bd2f99c5ddc4} | 0 ...range-10-3f2680208772a0e51aefc4ef5604dddf} | 0 ...range-11-f2ca12a948fd9b5b842168e7c7d7b768} | 0 ...range-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e} | 0 ...range-13-5baad22ed7efa18d73eb8349e57cf331} | 0 ...range-14-490d6253b73064ce403e4d04a8bc18f3} | 0 ...rrange-2-24ca942f094b14b92086305cc125e833} | 0 ...rrange-3-3b0f76816be2c1b18a2058027a19bc9f} | 0 ...rrange-4-86473a0498e4361e4db0b4a22f2e8571} | 0 ...rrange-5-d0ec6d66ff349db09fd455eec149efdb} | 0 ...rrange-6-cda81d86d127fca0e2fbc2161e91400d} | 0 ...rrange-7-5b13cfa4b730e38ef2794c1532968e04} | 0 ...rrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12} | 0 ...rrange-9-73b9ac83dbc9874dc9379ad4364d40ac} | 0 ...egation-0-739356d010a919138911f295fac81607 | 2 +- ...egation-2-f015c961b6c34ac56cb8fc52fb36d7c7 | 2 +- ...mat_mix-5-c2d0da9a0f01736a2163c99fc667f279 | 2 +- ...mat_mix-6-4b658b3222b7a09ef41d023215e5b818 | 2 +- ...ncefile-2-80ec34a069bc561aa6dc87314391b131 | 6 +- ...at_text-2-80ec34a069bc561aa6dc87314391b131 | 6 +- ...eaktask-3-550e9b759fb088a81feddeff2e4be64e | 2 +- ...ktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd | 2 +- ...ktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 | 2 +- ...ktask2-12-c608f51882260caec7bb9c57a0ba3179 | 2 +- ...ktask2-4-fe8b55e4e4098d7a2662338783a50306} | 0 ...ktask2-5-cf724251613216ec54f8ac2e6b9b92fd} | 0 ...ktask2-6-8c782ae8f8245bdbe90d068a6b577d1e} | 0 ...aktask2-8-604cf64d16b9d438ee187a10d9f6352e | 2 +- ...aktask2-9-6d551990a8a745dde1cb5244947b81a1 | 2 +- ...roupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby1-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d | 2 +- ...roupby1-5-dd7bf298b8c921355edd8665c6b0c168 | 2 +- ...roupby1-7-c2c54378ffce53ade73a9dda783903e7 | 2 +- ...oupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...oupby11-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...oupby11-6-686ee4a5b2e24e51ba1d41b42215accd | 2 +- ...oupby11-7-149d359546ab38226ffeb023d7414b3d | 2 +- ...upby12-0-67e864faaff4c6b2a8e1c9fbd188bb66} | 0 ...upby12-1-13ab74a58da514fe01dbeda0c3e79883} | 0 ...upby12-2-fd150794945892f3c926a1881cd819f4} | 0 ...upby12-3-8018bb917a0706925c14421ec2761663} | 0 ...oupby12-4-adc1ec67836b26b60d8547c4996bfd8f | 309 ++ ...1_limit-0-83c59d378571a6e487aa20217bd87817 | 2 +- ..._limit-3-607512a8dd16cd9ddde561eeabfa51db} | 0 ...1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 | 2 +- ...by1_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by1_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by1_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by1_map-6-adc1ec67836b26b60d8547c4996bfd8f | 309 -- ...by1_map-6-c83b156356c269b6aef263640a4f6b7b | 309 ++ ...p_nomap-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...p_nomap-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...ap_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 | 2 +- ...p_nomap-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 309 -- ...ap_skew-6-c83b156356c269b6aef263640a4f6b7b | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-c2c54378ffce53ade73a9dda783903e7 | 309 -- ..._noskew-6-f7eda18efd187ec4bf4cb135833746cb | 309 ++ ...roupby2-0-43d53504df013e6b35f81811138a167a | 2 +- ...roupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby2-2-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby2-6-41dfb7b036cae4972b275a0262ea2e4c | 2 +- ...2_limit-0-83c59d378571a6e487aa20217bd87817 | 2 +- ..._limit-1-a56d6499aef913e11ef599ac8b4f2a25} | 0 ...2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 | 5 - ...2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a | 5 + ...by2_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by2_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by2_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by2_map-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...by2_map-6-c83b156356c269b6aef263640a4f6b7b | 10 + ...istinct-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-232957c90d04497da7fe4e599e0dbb94 | 10 + ..._noskew-6-41dfb7b036cae4972b275a0262ea2e4c | 10 - ...istinct-6-41dfb7b036cae4972b275a0262ea2e4c | 10 - ...roupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby4-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby4-5-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...by4_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by4_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by4_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by4_map-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-2d1fb04c7060fafe470e14061b2a5b6f | 10 + ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...roupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby5-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby5-5-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...by5_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by5_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by5_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by5_map-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 309 -- ..._noskew-6-c83b156356c269b6aef263640a4f6b7b | 309 ++ ...roupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby6-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby6-5-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...by6_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by6_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by6_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by6_map-6-2d1fb04c7060fafe470e14061b2a5b6f | 10 + ...by6_map-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-2d1fb04c7060fafe470e14061b2a5b6f | 10 + ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-2d1fb04c7060fafe470e14061b2a5b6f | 10 + ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...roupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby7-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...roupby7-5-ed76c0068780120a6f23feefee303403 | 2 +- ...roupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...roupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...by7_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by7_map-1-7f98b724df05f51b3ec1f087a8da414e | 2 +- ...y7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...y7_map-10-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...y7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...y7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...by7_map-2-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by7_map-3-83c59d378571a6e487aa20217bd87817 | 2 +- ...by7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...by7_map-7-ed76c0068780120a6f23feefee303403 | 2 +- ...reducer-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...reducer-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...educer-10-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...educer-10-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...reducer-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...reducer-6-ed76c0068780120a6f23feefee303403 | 2 +- ...reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...reducer-9-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...p_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...p_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...ap_skew-6-ed76c0068780120a6f23feefee303403 | 2 +- ...ap_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...ap_skew-9-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-7f98b724df05f51b3ec1f087a8da414e | 2 +- ...noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...noskew-10-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ..._noskew-2-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-3-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ..._noskew-7-ed76c0068780120a6f23feefee303403 | 2 +- ...reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...reducer-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...educer-10-96d0598a2a4c17041a31b908d1f065e5 | 2 +- ...reducer-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...reducer-6-ed76c0068780120a6f23feefee303403 | 2 +- ...educer-8-ab2390a3b0ba76907a7bee390b2924a1} | 0 ...reducer-9-652510b8dc20117c65511f06e6e73d73 | 2 +- ...roupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby8-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...oupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...oupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...roupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...roupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...roupby8-8-7f98b724df05f51b3ec1f087a8da414e | 2 +- ...by8_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by8_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by8_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...by8_map-7-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...by8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...by8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...ap_skew-7-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...ap_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...ap_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ..._noskew-7-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ..._noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ..._noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...oupby9-10-7f98b724df05f51b3ec1f087a8da414e | 2 +- ...oupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...oupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...oupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...oupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...oupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...oupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...roupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...roupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...roupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...roupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...map_ppr-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...map_ppr-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...map_ppr-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...map_ppr-6-c83b156356c269b6aef263640a4f6b7b | 10 + ...istinct-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...istinct-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...stinct-10-6aa66df624cd32601218eee200977ce6 | 2 +- ...stinct-11-59a77127c166fef57504c011ccb427a6 | 2 +- ...istinct-5-6aa66df624cd32601218eee200977ce6 | 2 +- ...istinct-6-59a77127c166fef57504c011ccb427a6 | 2 +- ...istinct-7-293182ac89effb268855f5ac53e1ec11 | 2 +- ...educer-10-3b3ef4d975716744c85af560aa240abd | 10 - ...educer-13-38f492067df78144c272bb212633cc5e | 5 - ...educer-14-d7b3676444a9e95811184637dd0b3231 | 5 - ...educer-15-3b3ef4d975716744c85af560aa240abd | 10 - ...educer-16-b21ae88a5fc7f9b5c25ca954889ee421 | 10 - ...educer-17-7f344c2f066d74ecd135c43d39658bae | 32 - ...reducer-8-38f492067df78144c272bb212633cc5e | 5 - ...reducer-9-d7b3676444a9e95811184637dd0b3231 | 5 - ...educer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 | 2 +- ...educer2-5-376542befbcab97d864e874251720c40 | 2 +- ...educer2-6-8566e4c94d34b0fc4a71d34e89529c74 | 2 +- ...ducer3-10-73819ea1a7c0653a61652b3766afb003 | 2 +- ...ducer3-13-521e0c1054cfa35116c02245874a4e69 | 2 +- ...ducer3-14-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...ducer3-17-521e0c1054cfa35116c02245874a4e69 | 2 +- ...ducer3-18-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...educer3-4-521e0c1054cfa35116c02245874a4e69 | 2 +- ...educer3-5-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...educer3-8-521e0c1054cfa35116c02245874a4e69 | 2 +- ...educer3-9-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...g_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb | 2 +- ...g_float-1-4a435e268a327404f75725b82a32ee03 | 2 +- ...osition-0-422c2068a838f59324c1d9861225c824 | 1 - ...sition-10-b4c6c0aedcc924e2af65549b87b3f3f7 | 13 - ...sition-12-5583e5cfcf8083d45a3bd80434c1829f | 14 - ...sition-14-6f54558123eb5d63eeeb3e261c1dc5fb | 3 - ...osition-5-e2470670c5d709001fb17ecbc6e4f85d | 13 - ...osition-6-b4c6c0aedcc924e2af65549b87b3f3f7 | 13 - ...osition-9-e2470670c5d709001fb17ecbc6e4f85d | 13 - ...pby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...pby_ppr-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...pby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...istinct-5-adc1ec67836b26b60d8547c4996bfd8f | 10 - ..._sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ..._sort_1-1-ffe97dc8c1df3195982e38263fbe8717 | 1 - ...sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-14-bbdd53118f788d7bb679d094c41243c8 | 6 - ...sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ..._sort_1-2-fc30020d09151dc29be807795ad9475e | 1 - ...sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-24-d53196339980a00a619788bd799a32e7 | 5 - ...sort_1-28-c4ec0433a832ef551d70254957e3afca | 6 - ..._sort_1-3-fffea659b633b1f269b38556a7f54634 | 1 - ...sort_1-31-d53196339980a00a619788bd799a32e7 | 5 - ...sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd | 10 - ...sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd | 10 - ...sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-52-c4ec0433a832ef551d70254957e3afca | 6 - ...sort_1-56-5373c5449884d95bc7db9dab55239a49 | 6 - ...sort_1-59-c4ec0433a832ef551d70254957e3afca | 6 - ...sort_1-62-c4ec0433a832ef551d70254957e3afca | 6 - ...sort_1-63-dbcec232623048c7748b708123e18bf0 | 1 - ...sort_1-64-7f98b724df05f51b3ec1f087a8da414e | 1 - ...sort_1-65-83c59d378571a6e487aa20217bd87817 | 1 - ...sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 - ...sort_1-69-ed76c0068780120a6f23feefee303403 | 1 - ...sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 | 5 - ...sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd | 6 - ...sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 | 1 - ...sort_1-77-6d6eac558569563cc47a657daffa8ba7 | 2 - ...sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...sort_10-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...ort_10-10-b4e225bc4787d7718bae6f00d8addfe2 | 2 +- ...sort_10-2-fc30020d09151dc29be807795ad9475e | 2 +- ...sort_10-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 | 2 +- ...ort_11-11-d9bf7e80b71121935ed4b008ae916cb1 | 6 - ...ort_11-13-c7d70702783eb90d4f53028a63c318f8 | 6 - ...ort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 | 1 - ...ort_11-20-bd99462ed878bf4bec74b3cb9132908d | 1 - ...sort_11-7-bd99462ed878bf4bec74b3cb9132908d | 1 - ...sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 | 1 - ..._sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_2-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 | 2 +- ..._sort_2-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_2-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_2-5-c0f14def6a135cc50cba364e810ce28e} | 0 ..._sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_3-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_3-10-4bf8dba6e66e733423a3408d61897926 | 2 +- ...sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f | 2 +- ..._sort_3-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_3-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_3-5-c0f14def6a135cc50cba364e810ce28e} | 0 ..._sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_4-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd | 2 +- ...sort_4-14-75d59344b6114c0bb20d5eac301c2170 | 2 +- ..._sort_4-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_4-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_4-5-c0f14def6a135cc50cba364e810ce28e} | 0 ..._sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_5-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_5-10-4bf8dba6e66e733423a3408d61897926 | 2 +- ...ort_5-13-c0f14def6a135cc50cba364e810ce28e} | 0 ...sort_5-17-4bf8dba6e66e733423a3408d61897926 | 2 +- ..._sort_5-2-fc30020d09151dc29be807795ad9475e | 2 +- ...ort_5-20-c0f14def6a135cc50cba364e810ce28e} | 0 ...sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f | 2 +- ..._sort_5-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_5-5-c0f14def6a135cc50cba364e810ce28e} | 0 ..._sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_6-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd | 2 +- ..._sort_6-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_6-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_6-9-591e03d1cfc10821a601498df1ed6675} | 0 ..._sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_7-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_7-10-4bf8dba6e66e733423a3408d61897926 | 2 +- ..._sort_7-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_7-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc} | 0 ..._sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_8-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_8-11-bd99462ed878bf4bec74b3cb9132908d | 2 +- ..._sort_8-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_8-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc} | 0 ..._sort_8-8-bd99462ed878bf4bec74b3cb9132908d | 2 +- ..._sort_8-9-8e06b51e940e956f14a8c7679c3d423a | 2 +- ..._sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_9-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._sort_9-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_9-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc} | 0 ...y_sort_9-9-feec69facdc973a0ff78455f766845c | 2 +- ..._skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ..._skew_1-1-ffe97dc8c1df3195982e38263fbe8717 | 1 - ...skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-15-bbdd53118f788d7bb679d094c41243c8 | 6 - ...skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ..._skew_1-2-fc30020d09151dc29be807795ad9475e | 1 - ...skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-25-d53196339980a00a619788bd799a32e7 | 5 - ...skew_1-29-c4ec0433a832ef551d70254957e3afca | 6 - ..._skew_1-3-fffea659b633b1f269b38556a7f54634 | 1 - ...skew_1-32-d53196339980a00a619788bd799a32e7 | 5 - ...skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd | 10 - ..._skew_1-4-c67a488530dc7e20a9e7acf02c14380f | 1 - ...skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd | 10 - ...skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-53-c4ec0433a832ef551d70254957e3afca | 6 - ...skew_1-57-5373c5449884d95bc7db9dab55239a49 | 6 - ...skew_1-60-c4ec0433a832ef551d70254957e3afca | 6 - ...skew_1-63-c4ec0433a832ef551d70254957e3afca | 6 - ...skew_1-64-dbcec232623048c7748b708123e18bf0 | 1 - ...skew_1-65-7f98b724df05f51b3ec1f087a8da414e | 1 - ...skew_1-66-83c59d378571a6e487aa20217bd87817 | 1 - ...skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 - ...skew_1-70-ed76c0068780120a6f23feefee303403 | 1 - ...skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 | 5 - ...skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd | 6 - ...skew_1-75-6f7caef1c773268350c9cf94ad85be01 | 0 ...skew_1-76-4931c5a72a5231f67317d27ca025bb97 | 0 ...skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 | 1 - ...skew_1-78-6d6eac558569563cc47a657daffa8ba7 | 2 - ..._skew_1-8-c0ea81b686236d661166912040a16ea7 | 0 ...t_skew_1-9-f0ee61903aeacb758e2eada242e5e14 | 0 ..._test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._test_1-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._test_1-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._test_1-3-fffea659b633b1f269b38556a7f54634 | 2 +- ..._test_1-4-8e06b51e940e956f14a8c7679c3d423a | 2 +- ..._test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...test_1-6-c0f14def6a135cc50cba364e810ce28e} | 0 .../hash-0-a658b129316d666d4b01c1581eed1c1f | 2 +- ...text_cs-0-e319c8574a6cd8739e5fd5984ceed3cf | 0 ...text_cs-1-de3aa1c4674fb166b825b1e2f58d1950 | 0 ...reation-0-f880114c33c99a5f23c1465fd88f0db3 | 0 ...reation-1-a8bc76559014d9cdf07184208d582d25 | 0 ...eation-10-4bd6c86cd3f6a94964b3d13235e8e261 | 5 - ...eation-11-b5b7e4f7af5186033be12a4393dc3bb7 | 0 ...eation-12-9cc02e06c6051810c50e225bb2c66669 | 5 - ...eation-13-9c0ec67e79a0d50b46bd5c944c710fc4 | 0 ...eation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 | 5 - ...reation-15-7d7e1a479e7dcd8f9d4199775e05bc1 | 0 ...eation-16-ffa6d3fcef97b7322dd3759d4a70881d | 5 - ...eation-17-18ed0b70b0b6d076b96c9f73bfa721ad | 0 ...eation-18-bcdb19db031341c4a50264ccf49328e4 | 5 - ...eation-19-98dbf83283f9e073e88ba770ec5a707f | 0 ...creation-2-9d5d11cb38f2e097f16d2db5693f4f1 | 0 ...eation-20-68c5c98581c683b17ceaf1c0fd192871 | 5 - ...eation-21-4c8f6b48c437bf0be109fc0be1dc840e | 0 ...reation-22-f36cb2eed39691ca949b25182e2dd31 | 5 - ...eation-23-e7f21f556d3aa2bedb0717a167720741 | 0 ...eation-24-8cb102bd5addf7fece0e2691468bc3bf | 5 - ...eation-25-33c48966230b934ae8ddf74ff18bb9ca | 0 ...eation-26-f85db55b3f63ae186a1b6d5cec545939 | 0 ...eation-27-e4856f13692e63d61f72aaf75e50e5f1 | 0 ...eation-28-bd20d4b59e6489082a92fcbfcc5f8dbe | 0 ...eation-29-ee8d287111069805c41c9c0032adc46f | 0 ...reation-3-14b999fc6dfb10a3632afe14e08003e1 | 0 ...eation-30-f880114c33c99a5f23c1465fd88f0db3 | 0 ...eation-31-a8bc76559014d9cdf07184208d582d25 | 0 ...reation-32-9d5d11cb38f2e097f16d2db5693f4f1 | 0 ...eation-33-14b999fc6dfb10a3632afe14e08003e1 | 0 ...eation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 | 0 ...eation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 | 0 ...eation-36-21bcf37075b02097f16c8fc8130a83b8 | 0 ...eation-37-9334418431eca405f13206bd8db42a1b | 0 ...eation-38-f1f56119aede4f42221a68f6aaa42a26 | 0 ...eation-39-489b4ceb2f4301a7132628303f99240d | 1 - ...reation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 | 0 ...reation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 | 0 ...reation-6-21bcf37075b02097f16c8fc8130a83b8 | 0 ...reation-7-9334418431eca405f13206bd8db42a1b | 0 ...reation-8-f1f56119aede4f42221a68f6aaa42a26 | 0 ...reation-9-bf40d4d50d050f2f8342c07f5a9dcf0c | 0 ...x_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 | 0 ..._serde-0-db64b724719d27c7f0db4f51f5c4edaa} | 0 ..._serde-1-6560d12b69d55e5297a145ebc4bb0cb3} | 0 ...x_serde-1-f92d6c66d21791c11d2a822df04c1b63 | 3 - ..._serde-10-123301a057d4a46072d0431e00e20c4b | 5 - ...serde-10-c85e061ea9c5b90ca69b7450faad14b6} | 0 ..._serde-11-123301a057d4a46072d0431e00e20c4b | 5 + ..._serde-11-309e916d683a1a12ab62565697cb0046 | 0 ...serde-12-309e916d683a1a12ab62565697cb0046} | 0 ...x_serde-12-d590fd7cb9d433143de490d75686dd4 | 0 ..._serde-13-d590fd7cb9d433143de490d75686dd4} | 0 ...x_serde-2-f92d6c66d21791c11d2a822df04c1b63 | 3 + ...ex_serde-2-fd1b220f4eafb0ba9b519a156e3c87c | 0 ...x_serde-3-afcf2a156ccd4f79a0489b4593908d79 | 0 ..._serde-3-ebab588c84a7a29f03b41dcd98132229} | 0 ..._serde-4-afcf2a156ccd4f79a0489b4593908d79} | 0 ...x_serde-4-d7547751c37375a9238043bbe250e716 | 5 - ...x_serde-5-d7547751c37375a9238043bbe250e716 | 5 + ...x_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 | 0 ...x_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae | 1 - ..._serde-6-e6ff4b23b7f102e359afb4d53a1dedc3} | 0 ...x_serde-7-3b03210f94ec40db9ab02620645014d1 | 1 - ..._serde-7-c9d7dcde469d3b9a66965a64dd15e4ae} | 0 ...x_serde-8-35f48c7d6fa164bb84643657bc9280a8 | 1 - ..._serde-8-3b03210f94ec40db9ab02620645014d1} | 0 ..._serde-9-35f48c7d6fa164bb84643657bc9280a8} | 0 ...x_serde-9-c85e061ea9c5b90ca69b7450faad14b6 | 0 ...st_type-0-e231c5154b18cbc0baa082a7461dd13e | 0 ...st_type-1-c836a98522549d2a3fd43998afd8ae94 | 0 ...st_type-2-d8590c7336ae771b7a685bb544e8d2bd | 0 ...nerjoin-0-43d53504df013e6b35f81811138a167a | 2 +- ...nerjoin-4-35c7611180562dcd9dab834f41654095 | 2 +- ...tdriver-1-b2f337566a5075f3e3e81335008d95d3 | 4 +- .../input-1-6558e385bb08991302a72076dd7b7ff5 | 2 +- .../input0-1-efefd4364cd2790447fb0f908e87501f | 2 +- .../input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 | 17 - .../input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 | 4 +- ...input10-1-6970b6d2d451612b59fccbfd7ec68f74 | 17 - ...input10-2-73f00da5cfc254745d1d80f913eb6449 | 12 +- ...input11-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...1_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 | 2 +- ...input12-0-9b141c1e5917ca82c6bc36a9a2950a1e | 2 +- .../input12-1-2b9ccaa793eae0e73bf76335d3d6880 | 2 +- ...nput12-10-4d9eb316259a8e7ed6627bc27a639f7c | 2 +- ...input12-2-bab89dfffa77258e34a595e0e79986e3 | 2 +- ...input12-8-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...input12-9-3d08dc27c1a133c2497fc554c0d169bd | 2 +- ...input14-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...4_limit-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...4_limit-1-be9934fc5e6ecb9854eb7531a5929dcf | 0 ...4_limit-2-780cdc89e0e736790124b6bdac827951 | 0 ...4_limit-3-adc1ec67836b26b60d8547c4996bfd8f | 5 - ...input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 | 4 +- ...ut16_cc-1-5180e975a6babd51752706f1799e7df5 | 0 ...input17-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...input17-1-be9cde5e769f171f60f61a7739de8f17 | 0 ...input17-2-21166e268096f6ec67f4f57ec333e901 | 0 ...input17-3-adc1ec67836b26b60d8547c4996bfd8f | 11 - .../input19-1-f2832e249ab28bb3fb8e472012c5ffc | 0 ...nput19-1-f32df514de8156b5f5b435eea2c9be40} | 0 ...input19-2-5a804e02e4419e785d15e7f39d6c5730 | 2 +- ...1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 | 2 +- ...1_limit-5-eaaf713833e28a803c798562c7d6cd23 | 2 +- .../input2-1-e0efeda558cd0194f4764a5735147b16 | 4 +- .../input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd | 4 +- .../input2-4-235f92683416fab031e6e7490487b15b | 6 +- .../input2-5-9c36cac1372650b703400c60dd29042c | 2 +- .../input2-7-9c36cac1372650b703400c60dd29042c | 2 +- .../input2-9-48bf8c06ed0a264d0863613fe79795e1 | 17 - ...input21-1-70b803742328eacc69eb1ed044a5c6b8 | 0 ...nput21-1-c45ad493e95150b580be778da6065f36} | 0 ...input21-3-9809b74435cbaedef0dc6e6b88b180fe | 2 +- ...nput22-1-b663ec84da3f9d9b9594ea2da81b1442} | 0 ...input22-1-b7f46eccd104e6ed1b29e2de45089f66 | 0 ...input22-3-8285c1934441b12f6c016e13cb410e79 | 2 +- ...input24-3-3189f3b2990de94619b9cb583d9dd3c5 | 2 +- ...input26-1-8272225744e83ba4cbe158a5c113fce3 | 2 +- ...2_limit-1-fed7e0bb996623da7dd17793e835f785 | 2 +- .../input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d | 4 +- ...input3-10-10a1a8a97f6417c3da16829f7e519475 | 8 +- ...input3-11-9c36cac1372650b703400c60dd29042c | 2 +- ...input3-12-a22d09de72e5067a0a94113cdecdaa95 | 14 - ...input3-14-efee6816e20fe61595a4a2a991071219 | 6 +- .../input3-3-1c5990b1aed2be48311810dae3019994 | 6 +- .../input3-4-9c36cac1372650b703400c60dd29042c | 2 +- .../input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 | 14 - .../input3-7-1c5990b1aed2be48311810dae3019994 | 8 +- .../input3-8-4dc0fefca4d158fd2ab40551ae9e35be | 14 - ...input30-0-582c5fcbe2fe12cc8e7b21225583d96c | 0 ...input30-1-90c0d1a75de78c405413fd627caea4ab | 0 .../input30-2-823920925ca9c8a2ca9016f52c0f4ee | 1 - ...input30-3-c21dba410fb07a098f93430a9d21df79 | 1 - ...input30-4-f0ebd08e7675b19ae831824ef4d9e223 | 0 ...input30-5-38734677c27b5f90a8df5da6e6351c76 | 0 .../input30-6-f120ac8c87db9eebb5da7ac99b48600 | 1 - ...input30-7-95d10d459c088d5fbefd00bdd8d44c3f | 1 - .../input31-2-705764f8f7cab9378964af30b83f7fe | 0 ...input31-3-50c905261882f7fd8539fdd91e68151f | 0 ...input31-4-2f886fa357df9342733551fa1b53f913 | 0 ...input32-3-b0070890240c15d647af59f41b77ba3d | 0 ...input32-4-d0e1378a30e21e0198e47d9b668ee1f6 | 0 ...input32-5-8789d32fc5b784fe2d171566732c573e | 0 ...input37-0-86e2e274650fb56651607ea10d356fc0 | 0 ...input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 | 0 ...input39-0-7bd12162381231be9d578797818957a7 | 0 ...input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 | 0 ...nput39-10-28bf1b34c04a048da339eddd4c1fd779 | 0 ...nput39-11-6d0814cbb72eb96bfc75c95d06f1e528 | 1 - ...input39-12-f120ac8c87db9eebb5da7ac99b48600 | 1 - ...nput39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 | 1 - ...nput39-14-bcc1d5fd287f81bac1092a913b09956d | 1 - ...input39-2-ee667c6ab003bb83f6bf1c72153eba39 | 0 ...input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e | 0 ...input39-4-53453776bf062f28d371fc7336b7eae2 | 0 .../input39-5-823920925ca9c8a2ca9016f52c0f4ee | 1 - ...input39-6-763ab5853bff619e6525c01e46b2a923 | 1 - ...input39-7-9b141c1e5917ca82c6bc36a9a2950a1e | 1 - .../input39-8-2b9ccaa793eae0e73bf76335d3d6880 | 1 - ...input39-9-bab89dfffa77258e34a595e0e79986e3 | 1 - ...adoop20-0-4c33233bafd910d69c2b8845d35abffe | 0 ...adoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 | 0 ...doop20-10-6d0814cbb72eb96bfc75c95d06f1e528 | 1 - ...adoop20-11-f120ac8c87db9eebb5da7ac99b48600 | 1 - ...doop20-12-a6a77ae749a7e7f8022d71c03322fc21 | 1 - ...adoop20-2-ee667c6ab003bb83f6bf1c72153eba39 | 0 ...adoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e | 0 ...adoop20-4-53453776bf062f28d371fc7336b7eae2 | 0 ...hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee | 1 - ...adoop20-6-763ab5853bff619e6525c01e46b2a923 | 1 - ...adoop20-7-db1cd54a4cb36de2087605f32e41824f | 1 - ...adoop20-8-bab89dfffa77258e34a595e0e79986e3 | 1 - ...adoop20-9-28bf1b34c04a048da339eddd4c1fd779 | 0 ...3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 | 0 ...3_limit-1-7b46b8baf9c7628da9c190c96b917057 | 0 ...3_limit-2-c6583bdb759c8a050238a32a6ce8273d | 0 ...3_limit-3-2a87d8faa18a6311376812bd0453fece | 0 ...3_limit-4-70dad45d534146923fce88b2ffb99b0d | 0 ...3_limit-5-3664b564747487df13a5d109837219b5 | 0 ...3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 | 20 - ...input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e} | 0 .../input4-1-c139adc70f9942e527142e3be7fd2b87 | 27 - ...input4-2-b663ec84da3f9d9b9594ea2da81b1442} | 0 .../input4-2-b7f46eccd104e6ed1b29e2de45089f66 | 0 .../input4-4-271b04e0fbee2ee81bae21dcb46d55e4 | 2 +- ...nput40-1-a8adb8ae1d13607851431a1baf7578ba} | 0 ...input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 | 0 ...input40-2-e7ab3c9244fcfda76061b4153d796e93 | 2 +- ...input40-4-f241eafbb8d5da3f9c1737aed7b4f94e | 0 ...nput40-4-fdeea6b676c670b17c8d91e24a97a127} | 0 ...input40-5-ccdc9c87a8fdbeee0fa48927f9700361 | 2 +- ...nput40-6-6651f53efc5d03ed2d43b9d7aecc0002} | 0 ...input40-6-93a0c37189dfe2318ba6ad84616f0d64 | 0 ...input40-7-ccdc9c87a8fdbeee0fa48927f9700361 | 2 +- ...input41-0-763ab5853bff619e6525c01e46b2a923 | 2 +- ...input41-3-526399455dc5ecd4ea9f676b09fafeee | 2 +- ...input43-0-2baba8070f3585debc14b6bb3c83607a | 0 ...input46-0-b0cdbecce0321ac452c8e13e1bfc6924 | 0 ...input46-1-1efdd5ebfa732abdedeb10467ca71f7f | 0 ...input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca | 0 ...input46-3-c185163787977498a4b84f39f983c431 | 0 ...input46-4-7f05cde078d90c25780a7d5491b20c27 | 0 ...input46-5-f5c502e88a3dc3edb37b04af7d7955ab | 0 ..._delim-1-353d2238b781a117888a67bb7b2b2537} | 0 ...b_delim-1-5692d0e91dd0114729b8eb3aee388b72 | 0 ...b_delim-2-e52787bf798a941c854eb09d75efe617 | 2 +- ...4_limit-0-4f0124854141b8be1defa7a6d0877d8d | 0 ...t4_limit-1-c634fc723fb3aac3ce007069bdcb2af | 10 - .../input5-0-659e06570690cceeb3f37e10e855d2ea | 0 .../input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 | 0 .../input5-2-3d6eb15b4fe23d0a1aa303da818d97ad | 0 .../input5-3-adc1ec67836b26b60d8547c4996bfd8f | 11 - .../input6-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../input7-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../input8-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...arserde-0-df919fd41f281bf7b45a2340d0c9d43e | 0 ...narserde-1-2db9e6115d209fabcb0c06e5e666fa3 | 0 ...arserde-2-ac60752a883d3204c215fa01811701be | 0 ...arserde-3-3455e6f385b0f60be5d0c842ade0f1d7 | 11 - ...arserde-4-c471b057cdb4f3c3405b311ea2b92aa3 | 11 - ...icserde-0-92c95af00fd419aa106571f72fcad67d | 0 ...icserde-1-39bc8a2aea379563a62a465cc54aecbc | 0 ...icserde-2-7bee1cc403a04b53d0a0324819e1d768 | 0 ...icserde-3-adc1ec67836b26b60d8547c4996bfd8f | 11 - ...icserde-4-8bf42e02f82b0ac58e7d0b525a993b31 | 11 - ...zyserde-0-92c95af00fd419aa106571f72fcad67d | 0 ...zyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 | 0 ...yserde-10-d915efbc5233a49f48e81e74eef2a3c8 | 0 ...yserde-11-6aaa88142e86a9a39e980caed145e32c | 0 ...yserde-12-abde20a4a37ed330bc7128576dd18d7c | 11 - ...zyserde-2-7a165139976654640c86db8e4e5871cc | 0 ...zyserde-3-bdf93641b97ab6347ece67e2fb636e97 | 11 - ...zyserde-4-8cc058fb7986f59965976cad813267db | 11 - ...zyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ...azyserde-6-42e2838ee69484bf5301475905cee12 | 0 ...zyserde-7-b44a44827ad0dce044badd6b258eabd5 | 0 ...zyserde-8-abde20a4a37ed330bc7128576dd18d7c | 11 - ...zyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ...t_limit-1-77d7012bc901b0a9bcf9dae500ae2034 | 2 +- ...t_part0-1-1aa1486a207bedc275035acc3b37cbdb | 2 +- ...t_part1-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._part10-3-48b242bc305c9bf879e083fa11edc967 | 14 +- ..._part10-4-d0ba28297a8b73569d93605aa890aa09 | 2 +- ...t10_win-3-48b242bc305c9bf879e083fa11edc967 | 14 +- ...t10_win-4-d0ba28297a8b73569d93605aa890aa09 | 2 +- ...t_part2-4-93c97e1760e0d41b3791d6f08010a665 | 2 +- ...t_part2-5-9d0992a91951e6e4242b5b4c38d4e861 | 2 +- ...t_part3-1-ba5256285fb22a43b491253a2d519730 | 2 +- ...t_part5-3-e4419c33287ca1f48a43f61cca5b5928 | 2 +- ...t_part7-1-affad4cedcd29bb136bc477fc07e6ea0 | 2 +- ...t_part8-1-60b409a520999ba50e8b7c6e30de3474 | 2 +- ...t_part9-1-e60c60afc073367464898b8396e8f643 | 2 +- ...ncefile-0-68975193b30cb34102b380e647d8d5f4 | 2 +- ...ncefile-1-1c0f3be2d837dee49312e0a80440447e | 2 +- ...ncefile-5-3708198aac609695b22e19e89306034c | 2 +- ...ncefile-6-6716fa5aec125f0f8e9520e4318f71b9 | 2 +- ...stxpath-0-3c8a098a179d578119f75e5d7b214bd5 | 0 ...stxpath-1-ac18d29e8bd8aeed00296992edc17c0f | 0 ...estxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 | 0 ...stxpath-3-adc1ec67836b26b60d8547c4996bfd8f | 11 - ...txpath2-0-a1baa22f15f745a2dfe27ce52d363704 | 0 ...txpath2-1-27b77465d23aebf66a50c8074a75b755 | 0 ...txpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd | 0 ...txpath2-3-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...txpath3-0-f05134d54292acd1f2067027889a4fac | 0 ...txpath3-1-807b097ac2f785f774db03069ebbde11 | 11 - ...txpath4-0-73819ea1a7c0653a61652b3766afb003 | 1 - ...txpath4-1-f746888141a38ba707fad01d86d41960 | 0 ...txpath4-2-e4beab1294dcac60ff36e4f2561810b9 | 10 - ...txpath4-3-ae225e86c2ae20519ffdf23190454161 | 1 - ...txpath4-4-f746888141a38ba707fad01d86d41960 | 0 ...txpath4-5-e4beab1294dcac60ff36e4f2561810b9 | 10 - ...putddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 | 12 +- ...putddl3-2-73f945a673d2e388847c317f683f160c | 4 +- ...putddl4-1-dd94576788fa50ebcf950cdf837fbcf6 | 20 +- ...putddl4-2-7fdf00ff5c22ae284728e0f035396865 | 22 +- ...utddl6-1-572c1abb70f09726d1ba77bdc884597b} | 0 ...putddl6-1-8b96b4fba4cf54c399a008d0f3be1edf | 0 ...utddl6-2-3a4def4b370f75c5fcc1174626490363} | 0 ...putddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c | 0 ...putddl6-3-e81b962986706e1c16f059b407e3f05c | 10 +- ...putddl6-4-5855e2998e26f63e927854afa86c1f03 | 10 +- ...putddl6-5-47e6a8d33932cb014830812e1f7b1f94 | 2 +- ...putddl6-7-47e6a8d33932cb014830812e1f7b1f94 | 2 +- ...putddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 | 19 - ...utddl7-1-7195712efb4910294f63303ebce24453} | 0 ...putddl7-1-7b46b8baf9c7628da9c190c96b917057 | 0 ...utddl7-10-4eec8cef401b6bba00457dfbec58cc2d | 0 ...tddl7-10-7c9248b56948716913d332bd712d69bd} | 0 ...utddl7-11-6e1f1709d770ed76aee6ff5f76856e63 | 2 +- ...putddl7-12-2c56d4a781242b0521f82bb0d2cd277 | 4 +- ...utddl7-13-45059a21c202b4658285738ee62a018a | 4 +- ...utddl7-14-30c87bc734c2afa4fea0facdf7279145 | 8 +- ...putddl7-15-fed9badc255db68c3ed1cd1446d9fbe | 8 +- ...putddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 | 2 +- ...utddl7-4-68715ba2c11220be62394c86453e6d54} | 0 ...putddl7-4-7513658e8abe9d9b72c7219321c56fa9 | 0 ...putddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 | 2 +- ...putddl7-7-48640fff8428a0dc6e90a7243adaf730 | 0 ...utddl7-7-59dd2d2556769e19bdc0a444f40f8a71} | 0 ...putddl7-8-495dc87b0bde752c890f213ff9531508 | 2 +- ...putddl8-1-c70f2d2544633366b76b92bcff18e995 | 10 +- ...itions-3-86653b3af59df59f225ee00ff5fc119f} | 0 ...titions-3-b7aaedd7d624af4e48637ff1acabe485 | 0 ...titions-4-dece2650bf0615e566cd6c84181ce026 | 0 ...itions-4-e81d45a5bec5642ec4b762f1c1a482af} | 0 ...pressed-0-ea607fbed28d20e5726f4501285d698d | 2 +- ...pressed-4-5133d2457097962811a2adf0ecd9e4ef | 2 +- ...pressed-6-5133d2457097962811a2adf0ecd9e4ef | 2 +- ...pressed-8-5133d2457097962811a2adf0ecd9e4ef | 2 +- ...t_into1-0-ae5ea07929262bde22fbe7ebe80d4992 | 0 ...t_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b | 0 ..._into1-10-c260979323c1ebdf68c6fbe003d43792 | 0 ...t_into1-11-41015d6409c5ebf670eed4999157fdb | 1 - ..._into1-12-ae5ea07929262bde22fbe7ebe80d4992 | 0 ...t_into1-2-ff6a1b25c911def274921df1bae476b7 | 0 ...t_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 | 0 ...rt_into1-4-41015d6409c5ebf670eed4999157fdb | 1 - ...t_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 | 0 ...t_into1-6-4b2e32fe57b2297d22bccb4656bdee30 | 0 ...rt_into1-7-41015d6409c5ebf670eed4999157fdb | 1 - ...t_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d | 1 - ...t_into1-9-31eb4770dc60e4765065ac7f84811d1b | 0 ...t_into2-0-957c6402df0fd7d2fccbc688e49e9661 | 0 ...t_into2-1-9828bb831fd11667b48678e5952a0941 | 0 ..._into2-10-df53336f364fe09e9591e769c13b5519 | 0 ..._into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa | 0 ..._into2-12-4cf03cb3982a457f2f72220265ecc844 | 1 - ..._into2-13-957c6402df0fd7d2fccbc688e49e9661 | 0 ...t_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 | 0 ...t_into2-3-9d5556040de01fd02d5501d141effff2 | 0 ...t_into2-4-9d5556040de01fd02d5501d141effff2 | 0 ...t_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 | 1 - ...t_into2-6-4cf03cb3982a457f2f72220265ecc844 | 1 - ...t_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 | 0 ...t_into2-8-452111285dda40205ee587de8e972896 | 0 ...t_into2-9-4cf03cb3982a457f2f72220265ecc844 | 1 - ...t_into3-0-7e35c30348a53b0748bc1bb85f75c195 | 0 ...t_into3-1-f6dd7262d45a21781d85bc343ede8fb5 | 0 ..._into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 | 1 - ..._into3-11-9cfd2d054f84262eb74a870b6365db87 | 1 - ..._into3-12-7e35c30348a53b0748bc1bb85f75c195 | 0 ..._into3-13-f6dd7262d45a21781d85bc343ede8fb5 | 0 ...t_into3-2-e51c25bae2408422a56826a263479468 | 0 ...t_into3-3-c46699c465fefe6baab35499a32b452d | 0 ...rt_into3-4-e9f4f47686fe97482b0a769a15481dd | 0 ...t_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 | 0 ...t_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 | 1 - ...t_into3-7-9cfd2d054f84262eb74a870b6365db87 | 1 - ...t_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 | 0 ...t_into3-9-19d1be183f7985f7066f86572abc82c8 | 0 ...t_into4-0-b4831621b2a02fc4e8e655b03c289310 | 1 - ...t_into4-1-3d466d45197fcf1eff55d76ef0a29720 | 0 ..._into4-10-a8016ca816f7b362cf3a0384ca91b159 | 1 - ..._into4-11-bdbfaf06bc4e323863db65fd29882eaa | 0 ..._into4-12-49b12993ebc1647b9dca8b9af19deca9 | 0 ..._into4-13-a5bb41af609f38f01d8c71334cc8d71b | 1 - ..._into4-14-3d466d45197fcf1eff55d76ef0a29720 | 0 ..._into4-15-f6ad1a49459fb6cd232fccc4a6062b25 | 0 ...t_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 | 0 ...t_into4-3-43629aaa698fb6e2db4586124561e69b | 0 ...t_into4-4-8adba808fd505f4bf0ffcc61a618480a | 0 ...t_into4-5-6bc47894aa917051abb98d0b52f43881 | 0 ...t_into4-6-8c1683bee2927da76bb0dbf44a373738 | 0 ...t_into4-7-a8016ca816f7b362cf3a0384ca91b159 | 1 - ...t_into4-8-6bc47894aa917051abb98d0b52f43881 | 0 ...t_into4-9-8c1683bee2927da76bb0dbf44a373738 | 0 ...t_into5-0-9afa473f2111cf0d9ae62041bd97f840 | 0 ...t_into5-1-8fc8db6a5564324339192f23208ffc1c | 0 ...t_into5-10-375cae396c768c1efe5d17b9f5f45f8 | 0 ..._into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d | 0 ..._into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 | 1 - ..._into5-13-e06a0b7252278141d50466e08f15b391 | 0 ..._into5-14-a3a4c16b0b723f97f654b4b5e80575c2 | 0 ..._into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 | 1 - ..._into5-16-9afa473f2111cf0d9ae62041bd97f840 | 0 ...t_into5-2-548a3a10c15c333c63fc1c239ee8b62c | 0 ...t_into5-3-a4b25f172af356ec98035329b95ddbd3 | 0 ...t_into5-4-3d5343a79ee8b680f3b74b22db6658e6 | 0 ...t_into5-5-f382c5b31afe9e0251fa9457c46c12a5 | 0 ...t_into5-6-a058ba199b9777d48b6c6595f2388533 | 1 - ...t_into5-7-2004b4ecf6ceddb033727f8d5ebab42f | 0 ...t_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 | 0 ...t_into5-9-a058ba199b9777d48b6c6595f2388533 | 1 - ...t_into6-0-16367c381d4b189b3640c92511244bfe | 1 - ...t_into6-1-a4fb8359a2179ec70777aad6366071b7 | 1 - ..._into6-10-16500f4af2c8638a670e867e59f6d457 | 0 ..._into6-11-95297f4dcc4dd6a51de4785ccd25fbba | 0 ..._into6-12-29afaab2cf10eaa65afaa6c0fcca0902 | 1 - ..._into6-13-b357a845fb5f668b09dea94c6d2e7d66 | 2 - ..._into6-14-1f00d9cc34225cb358a54c6d4008cb47 | 0 ..._into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d | 0 ...t_into6-2-1f00d9cc34225cb358a54c6d4008cb47 | 0 ...t_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d | 0 ...t_into6-4-5f12794e99c74355a23d2fda9c7c170f | 0 ...t_into6-5-de641eb41a9100e755a9ae641c752b30 | 0 ...t_into6-6-f6e7141a435922193937aa10085b0656 | 0 ...t_into6-7-6826c610b8d04ab3464712bd9cddbcd3 | 0 ...t_into6-8-33ec9514947e0b737e957bdcbbd87573 | 0 ...t_into6-9-3ccb7bc735e406ad1723c758f01bcaab | 1 - ...ternal1-0-eb0745518b859c8497506a627bfd9860 | 0 ...ternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a | 1 - ...ternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b | 1 - ...ternal1-3-f64289fb03ab105e12659fc3972ca241 | 1 - .../join0-2-52055f2dce8eac4e5249d02c42c0da87 | 2 +- .../join1-0-43d53504df013e6b35f81811138a167a | 2 +- .../join1-4-35c7611180562dcd9dab834f41654095 | 2 +- .../join10-1-73136f8e6e9ba82f75570afd15c2828d | 2 +- .../join11-1-b580f87daf1763cd8c5e59ad5b271232 | 2 +- .../join12-1-496c8aabaf3261e8fefd7b357f2ac7f | 2 +- .../join13-1-696b36d15c4358145f77c2b15b7507d5 | 2 +- .../join14-1-9b141c1e5917ca82c6bc36a9a2950a1e | 2 +- .../join14-2-2b9ccaa793eae0e73bf76335d3d6880 | 2 +- .../join14-3-bab89dfffa77258e34a595e0e79986e3 | 2 +- .../join14-4-27f1a57fbb815d169af86ae2f8305cb6 | 2 +- .../join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 | 2 +- ...adoop20-1-db1cd54a4cb36de2087605f32e41824f | 2 +- ...adoop20-2-bab89dfffa77258e34a595e0e79986e3 | 2 +- ...adoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 | 2 +- .../join15-1-81d76d3bf59889b07b413b6f88772667 | 2 +- .../join17-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join18-1-49f8ba8d43a6bb1d892ba66a812304f5 | 2 +- .../join2-3-cac2c9e0f8601dd56822c990774e0696 | 2 +- .../join20-1-9685f2d31ffe922d3ea217de32ca3049 | 2 +- .../join20-3-3331a020843caccf2fa32a1225c5c3a9 | 2 +- .../join21-1-3536b7d78713e86ee67f5f6c9b88958f | 2 +- .../join23-1-91b8e7fe75a7e3ba8147c56734436681 | 2 +- .../join24-2-d79325ef6494aa87843fdfd78de7c812 | 2 +- .../join25-0-8934d9b821aa4b34b760f73eff56cd06 | 2 +- .../join25-4-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join26-3-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join27-3-e86808fdbd54120d1e2356f8f61c02f9 | 2 +- .../join28-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join28-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join28-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join28-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join29-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join29-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join29-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join29-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join3-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join30-3-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join31-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join31-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join31-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join31-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join32-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join32-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join32-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join32-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- ...ssSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a | 2 +- ...ssSize-13-ed70124968560328930327ecb108c4e9 | 2 +- ...ssSize-16-ed70124968560328930327ecb108c4e9 | 2 +- ...ssSize-19-ed70124968560328930327ecb108c4e9 | 2 +- ...essSize-2-24ca942f094b14b92086305cc125e833 | 2 +- ...ssSize-22-ed70124968560328930327ecb108c4e9 | 2 +- ...essSize-3-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...essSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 | 2 +- ...essSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a | 2 +- .../join33-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join33-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join33-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join33-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join34-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join34-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join34-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join34-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join35-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join35-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join35-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join35-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join36-0-8934d9b821aa4b34b760f73eff56cd06 | 2 +- .../join36-8-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join37-0-8934d9b821aa4b34b760f73eff56cd06 | 2 +- .../join37-4-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join38-2-eacdb3417c4a563982c488812d654c9 | 2 +- .../join38-4-53d219706847e890de1dcd369563ebef | 2 +- .../join39-3-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join4-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join40-0-d2b5e23edec42a62e61750b110ecbaac | 2 +- ...join40-10-1d1f79e8e773d630f355c1a89d84b5aa | 2 +- ...join40-12-aaddbef9069aba3ebeb310be74671cda | 2 +- .../join40-2-507b1d9f6abbdb756a589d7bc4826251 | 2 +- .../join40-4-61fb097bda1751910de966d6a4a8f0b7 | 2 +- .../join40-6-9685f2d31ffe922d3ea217de32ca3049 | 2 +- .../join40-8-3331a020843caccf2fa32a1225c5c3a9 | 2 +- .../join41-1-25e434b6d05e08fdd5f4d9957438917 | 2 +- .../join41-3-1dc98363e7da167dc45711a87ef3a988 | 2 +- .../join41-4-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join41-6-1dc98363e7da167dc45711a87ef3a988 | 2 +- .../join5-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join6-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join7-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join8-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join9-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...in_1to1-1-5bab379018a4fbef12cc93658f26580a | 0 ...n_1to1-1-789b2636cfb6a08965e0bd190e419762} | 0 ...n_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e | 2 +- ...n_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-12-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-13-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-15-5e48ba086f1376939535081b60f82727 | 2 +- ...n_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 | 2 +- ...n_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-18-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-19-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-21-5e48ba086f1376939535081b60f82727 | 2 +- ...n_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 | 2 +- ...n_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-24-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-25-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-27-5e48ba086f1376939535081b60f82727 | 2 +- ...n_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e | 2 +- ...n_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-3-6228e662e573a00ed04550d049d97a3b} | 0 ...in_1to1-3-ee6db4188755bf471a12316ec7301500 | 0 ...n_1to1-30-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-31-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-33-5e48ba086f1376939535081b60f82727 | 2 +- ...n_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 | 2 +- ...n_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-36-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-37-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-39-5e48ba086f1376939535081b60f82727 | 2 +- ...in_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 | 2 +- ...in_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...in_1to1-6-6081eb976b4aef2132418510756a385b | 2 +- ...in_1to1-7-281b888188eac90c4bf670417f25cc0c | 2 +- ...in_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...in_1to1-9-5e48ba086f1376939535081b60f82727 | 2 +- ..._array-2-16840a0266cad03a1a0b134d105b854f} | 0 ...n_array-2-a4363f7c7e4b7d717ed90e77c37581de | 0 ..._array-3-a6ca6b64324596831033fdfe5b63a942} | 0 ...n_array-3-ddd65703cdad8959cd0cd831304b0ab9 | 0 ...n_array-4-b235265cd6bd58fd743c27b02e547d62 | 2 +- ...n_array-5-a9b9419b94631f8fe1c2297ebf103a9a | 2 +- ...n_array-6-6eded94bd39189ea6d67fe383f9b865c | 2 +- ...nsitive-1-404d691e85c7b74bad73576ee80de290 | 0 ...sitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf} | 0 ...nsitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc | 0 ...nsitive-3-93300f4a9242fa2804a5b368538d83f} | 0 ...nsitive-4-c880b2256f97413b8fe68d19d99747fd | 2 +- ...n_empty-4-df2401785dfa257de49c3ad80b0f480a | 2 +- ...n_empty-5-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...filters-1-100faa7fd01bfb4390c782bb262a9403 | 0 ...ilters-1-bee6095f42de6a16708c2f9addc1b9bd} | 0 ...ilters-10-f0c5c51de4151a17a77c780be0c13e01 | 2 +- ...lters-100-a4c7cd87175222bea19fd33018890efe | 2 +- ...lters-101-f086409eb336282af5a00f4c7192ef2b | 2 +- ...lters-102-634888c4fa78ce10181c9514c6393554 | 2 +- ...lters-103-c020d3461658ae8e118281f40264ae5b | 2 +- ...lters-104-c9b79f30e1f25672ec89014f966b41b0 | 2 +- ...ilters-105-b3d9dcbb7e1caef97646eb89edf82eb | 2 +- ...lters-106-870999978978b22f21997899f1e652b8 | 2 +- ...lters-107-94824a62d882227f270a66ec7ef28cd4 | 2 +- ...lters-108-d793c1514545499f58fb1b355cbd3f0e | 2 +- ...lters-109-2709001b4aa57ed01ba975e83b556475 | 2 +- ...ilters-11-6fb35e1a65211e68de5df4a1cb9c7117 | 2 +- ...lters-110-86868ef0f28c5b42f04fb9ca64aaa3ba | 2 +- ...lters-111-fd3188d025e4c84d45cbb265d08ca569 | 2 +- ...lters-112-a0cd07949ff7dbc2287dc060bedb1942 | 2 +- ...ilters-113-48c4978768872751832149d72cdf0ba | 2 +- ...lters-114-58b8db46ea5080791c7416838d3e8f95 | 2 +- ...lters-115-98e45a3167d19d09219076a2d93afa49 | 2 +- ...ilters-116-bec2950fff2c1b3c4dc5be846653ea6 | 2 +- ...lters-117-ee8471d93c3778f4b48c5c59f473dd35 | 2 +- ...lters-118-9b395d7db8722a467d46588d0f27fc9e | 2 +- ...lters-119-7688784396db55ff148292a9755c918a | 2 +- ...ilters-12-4c29d6be8717481332cd1ee7ca17690e | 2 +- ...lters-120-6578651ffbf95c0b02825e2125e32709 | 2 +- ...lters-121-6dc4977da7f8dc7c636a03d7ad892ada | 2 +- ...lters-122-64bb9f9a0eef32d2392b80b93c9b2c98 | 2 +- ...filters-13-c6a291879bdb37f0c84f6074f257d52 | 2 +- ...ilters-14-ef8255dcad808f9128d79e6ee9f368cf | 2 +- ...ilters-15-a83678913c62249c8fdf2dac1f6e3046 | 2 +- ...ilters-16-a6511579b5e4c2d845664f0c4a6ce622 | 2 +- ...ilters-17-c22aab929016fa6f4b6d1e14cc3a4940 | 2 +- ...ilters-18-8ef51af7455eaeeaefff1c98e16dcc65 | 2 +- ...ilters-19-e164192e6538b428c8b53e008a978d3f | 2 +- ...filters-2-67fff9e6931a7320444e857e01b3d496 | 2 +- ...ilters-20-7a5da20822bf51ed69ccf640cbb816cf | 2 +- ...ilters-21-13d6d5335625fc3386a8011dc557002e | 2 +- ...ilters-22-f12ffd6cc9213d9c64cbea1b6db0632e | 2 +- ...ilters-23-a800b885358695758afdb719cdefa94f | 2 +- ...ilters-24-df3db5a91f3f4e88e18d2b1cc5b47113 | 2 +- ...ilters-25-435ecfc7f9cb5f32019fe498c21ccad3 | 2 +- ...ilters-26-da36fab3d1686aedd415a7a0f0253eca | 2 +- ...ilters-27-5f4a5437696f2a60bec9ac1443244242 | 2 +- ...ilters-28-2acf41a9f6efac0d800df557db716359 | 2 +- ...ilters-29-cf5fef4ddc313209f5ab1e5ea3763e35 | 2 +- ...filters-3-989b3dbd15ba601ae80fe454e03213d7 | 2 +- ...ilters-30-97f94f35ebc30f776a19bed59c7fb4bf | 2 +- ...ilters-31-4923f8ba429f0c71ad9b6b9499e73a7f | 2 +- ...ilters-32-5978cd7936c296493a16a31b926043ab | 2 +- ...ilters-33-607d64d50ef9aad424bd22b358efe027 | 2 +- ...ilters-34-35c2c8d74bc0ebc4710c81333cb254a9 | 2 +- ...ilters-35-a4c7cd87175222bea19fd33018890efe | 2 +- ...ilters-36-f086409eb336282af5a00f4c7192ef2b | 2 +- ...ilters-37-634888c4fa78ce10181c9514c6393554 | 2 +- ...ilters-38-c020d3461658ae8e118281f40264ae5b | 2 +- ...ilters-39-c9b79f30e1f25672ec89014f966b41b0 | 2 +- ..._filters-4-33bfcd576019d7e32683556f66e3757 | 2 +- ...filters-40-b3d9dcbb7e1caef97646eb89edf82eb | 2 +- ...ilters-41-870999978978b22f21997899f1e652b8 | 2 +- ...ilters-42-94824a62d882227f270a66ec7ef28cd4 | 2 +- ...ilters-43-d793c1514545499f58fb1b355cbd3f0e | 2 +- ...lters-46-268d8fb3cb9b04eb269fe7ec40a24dfe} | 0 ...ilters-46-aa161b0d9fe9d1aad10654fce0e3670b | 0 ...ilters-47-3c52df82c7d78501610f3f898103f753 | 0 ...lters-47-6dc6866a65c74d69538b776b41b06c16} | 0 ...ilters-48-1d85bb008e02ef4025171a4bc0866a6c | 0 ...lters-48-e884480a0f7273d3e2f2de2ba46b855c} | 0 ...lters-49-98fd86aea9cacaa82d43c7468109dd33} | 0 ..._filters-49-e79c906b894fed049ddfab4496a4e3 | 0 ...filters-5-f0c0d07019afb1bbe162e3183e18023e | 2 +- ...ilters-50-3e6612a89e9124592e790594775054b1 | 2 +- ...ilters-51-60a5f56f33fc8854a2b687005f0d96ac | 2 +- ...ilters-52-64cabe5164130a94f387288f37b62d71 | 2 +- ...ilters-53-2709001b4aa57ed01ba975e83b556475 | 2 +- ...ilters-54-86868ef0f28c5b42f04fb9ca64aaa3ba | 2 +- ...ilters-55-fd3188d025e4c84d45cbb265d08ca569 | 2 +- ...ilters-56-a0cd07949ff7dbc2287dc060bedb1942 | 2 +- ...filters-57-48c4978768872751832149d72cdf0ba | 2 +- ...ilters-58-58b8db46ea5080791c7416838d3e8f95 | 2 +- ...ilters-59-98e45a3167d19d09219076a2d93afa49 | 2 +- ...filters-6-c0c40d001cac0bc91095dddda1513ad9 | 2 +- ...filters-60-bec2950fff2c1b3c4dc5be846653ea6 | 2 +- ...ilters-61-ee8471d93c3778f4b48c5c59f473dd35 | 2 +- ...ilters-62-9b395d7db8722a467d46588d0f27fc9e | 2 +- ...ilters-63-7688784396db55ff148292a9755c918a | 2 +- ...ilters-64-6578651ffbf95c0b02825e2125e32709 | 2 +- ...ilters-65-6dc4977da7f8dc7c636a03d7ad892ada | 2 +- ...ilters-66-64bb9f9a0eef32d2392b80b93c9b2c98 | 2 +- ...ilters-67-67fff9e6931a7320444e857e01b3d496 | 2 +- ...ilters-68-989b3dbd15ba601ae80fe454e03213d7 | 2 +- ...filters-69-33bfcd576019d7e32683556f66e3757 | 2 +- ...filters-7-89963646509154a2fb1ddbbf1f55349d | 2 +- ...ilters-70-f0c0d07019afb1bbe162e3183e18023e | 2 +- ...ilters-71-c0c40d001cac0bc91095dddda1513ad9 | 2 +- ...ilters-72-89963646509154a2fb1ddbbf1f55349d | 2 +- ...ilters-73-69e0235472d7cee7d83037cd083544a5 | 2 +- ...ilters-74-b6372cc006844e8488a3b7836c67daaa | 2 +- ...ilters-75-f0c5c51de4151a17a77c780be0c13e01 | 2 +- ...ilters-76-6fb35e1a65211e68de5df4a1cb9c7117 | 2 +- ...ilters-77-4c29d6be8717481332cd1ee7ca17690e | 2 +- ...filters-78-c6a291879bdb37f0c84f6074f257d52 | 2 +- ...ilters-79-ef8255dcad808f9128d79e6ee9f368cf | 2 +- ...filters-8-69e0235472d7cee7d83037cd083544a5 | 2 +- ...ilters-80-a83678913c62249c8fdf2dac1f6e3046 | 2 +- ...ilters-81-a6511579b5e4c2d845664f0c4a6ce622 | 2 +- ...ilters-82-c22aab929016fa6f4b6d1e14cc3a4940 | 2 +- ...ilters-83-8ef51af7455eaeeaefff1c98e16dcc65 | 2 +- ...ilters-84-e164192e6538b428c8b53e008a978d3f | 2 +- ...ilters-85-7a5da20822bf51ed69ccf640cbb816cf | 2 +- ...ilters-86-13d6d5335625fc3386a8011dc557002e | 2 +- ...ilters-87-f12ffd6cc9213d9c64cbea1b6db0632e | 2 +- ...ilters-88-a800b885358695758afdb719cdefa94f | 2 +- ...ilters-89-df3db5a91f3f4e88e18d2b1cc5b47113 | 2 +- ...filters-9-b6372cc006844e8488a3b7836c67daaa | 2 +- ...ilters-90-435ecfc7f9cb5f32019fe498c21ccad3 | 2 +- ...ilters-91-da36fab3d1686aedd415a7a0f0253eca | 2 +- ...ilters-92-5f4a5437696f2a60bec9ac1443244242 | 2 +- ...ilters-93-2acf41a9f6efac0d800df557db716359 | 2 +- ...ilters-94-cf5fef4ddc313209f5ab1e5ea3763e35 | 2 +- ...ilters-95-97f94f35ebc30f776a19bed59c7fb4bf | 2 +- ...ilters-96-4923f8ba429f0c71ad9b6b9499e73a7f | 2 +- ...ilters-97-5978cd7936c296493a16a31b926043ab | 2 +- ...ilters-98-607d64d50ef9aad424bd22b358efe027 | 2 +- ...ilters-99-35c2c8d74bc0ebc4710c81333cb254a9 | 2 +- ...overlap-0-990e447b6447ced0d9684eb7db9e63ce | 0 ...overlap-1-a7336cd2d643f92715b42f9c6eb2c108 | 0 ...ive_626-3-2609f2809e0c6ae1ca93806e37960990 | 0 ...ve_626-3-4a2f2f2858540afea9a195b5322941ee} | 0 ...ive_626-4-387e3bc517b49d4e1c9752c07b72b790 | 0 ...ve_626-4-4bb73b33747da4ed852df381b7b45a71} | 0 ...ive_626-5-a0eb25c15b6ca1a04da14781b1213e11 | 0 ...ve_626-5-c6a3ae6f3539ab48b996060fb51d8ebe} | 0 ...ive_626-7-d9c817eebc533bedc3ef9172d325a2c2 | 2 +- ...ap_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- ...map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- ..._nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f} | 0 ...n_nulls-1-97641998eb9ddf2dff56de6758486aa0 | 0 ..._nulls-10-39071d8e6b246cfd405714dbf0b5337b | 2 +- ..._nulls-11-545dff878ea0d79cdfee517572c9e0c8 | 2 +- ..._nulls-12-5e1ca8627aa685435142d8e339e77062 | 2 +- ..._nulls-13-5f670a20cbec42a34685ee0398ad7e2d | 2 +- ...n_nulls-14-e97ba69145da387a4a66635b8499077 | 2 +- ..._nulls-15-e9f9be8be6e936fb832483dfa0be5126 | 2 +- ..._nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 | 2 +- ...in_nulls-17-2b0bfe5754456475ceb6af4694165f | 2 +- ..._nulls-18-321cf9d31dac835c3def6ca3b3b860a2 | 2 +- ..._nulls-19-739bf8e440e698540d18c29226c3564c | 2 +- ...n_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 | 2 +- ..._nulls-20-fff960f1cb832bc93e3d1de519e573d5 | 2 +- ..._nulls-21-62a25fb819ae5c1ea757b6e759082a2e | 2 +- ..._nulls-22-5b2df5518994ae86c041484561857da0 | 2 +- ..._nulls-23-982c3e55235cafa3d89b5dee4366fdf8 | 2 +- ..._nulls-24-86ad66543a735d396f7336cb5bdfa495 | 2 +- ..._nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 | 2 +- ..._nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 | 2 +- ..._nulls-27-6ee7affed896b1c539628ab081842b83 | 2 +- ..._nulls-28-455aace3472c5840a885b6fab6a046cb | 2 +- ..._nulls-29-2c278a88713aef5cd30ff3720ef3eeeb | 2 +- ...n_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 | 2 +- ..._nulls-30-2c0c41da38714d1b16feffb00fa08bb1 | 2 +- ..._nulls-31-a33c48d38817ee3a7aca511dc7793486 | 2 +- ..._nulls-32-e6b104ae96622ff75d510efc6efc9352 | 2 +- ..._nulls-33-1284a11bf6aeef8ff87b471d41985f26 | 2 +- ..._nulls-34-aeb90811861431cadc5512637793afc1 | 2 +- ..._nulls-35-2d1d18d4e9775ec69333947fbd62bc82 | 2 +- ..._nulls-36-7c029c91141b36af79ba0dc1de73a257 | 2 +- ..._nulls-37-fa84731f5a6beec20d64a7981815b9bc | 2 +- ...n_nulls-4-a1f20b4863428627bae1316755cc2d94 | 2 +- ...nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe} | 0 ..._nulls-40-aa161b0d9fe9d1aad10654fce0e3670b | 0 ..._nulls-41-3c52df82c7d78501610f3f898103f753 | 0 ...nulls-41-6dc6866a65c74d69538b776b41b06c16} | 0 ..._nulls-42-1d85bb008e02ef4025171a4bc0866a6c | 0 ...nulls-42-e884480a0f7273d3e2f2de2ba46b855c} | 0 ...nulls-43-98fd86aea9cacaa82d43c7468109dd33} | 0 ...in_nulls-43-e79c906b894fed049ddfab4496a4e3 | 0 ..._nulls-44-2db30531137611e06fdba478ca7a8412 | 2 +- ..._nulls-45-e58b2754e8d9c56a473557a549d0d2b9 | 2 +- ..._nulls-46-64cabe5164130a94f387288f37b62d71 | 2 +- ..._nulls-47-ebf794e8b51be738e2d664f249869de1 | 2 +- ...n_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 | 2 +- ..._nulls-49-2d20d8f4221281a6b1cd579e761f81b7 | 2 +- ...n_nulls-5-5ec6e3df7181e0738244623cc01bf22c | 2 +- ...n_nulls-50-8b26343e74c161cf74ece5f0bd21470 | 2 +- ..._nulls-51-75339d2eb2afabf5dd088074b2563d8f | 2 +- ...n_nulls-52-caad1db99085760daaf8f96c0ce5564 | 2 +- ..._nulls-53-ec965e66e7fad403cd0ea388c3e70363 | 2 +- ...in_nulls-54-6f15c794b5587430ebb685cc61c502 | 2 +- ..._nulls-55-a1c73326f8c8d9beccda3ba365352564 | 2 +- ..._nulls-56-f7b9629093c818b6c384c79d1458d178 | 2 +- ..._nulls-57-cf353446d7f358a508f17d0984b90158 | 2 +- ..._nulls-58-5f9a59160a76f9b649102a97987ed33a | 2 +- ..._nulls-59-8753a39412ac59c7a05951aeeea73b24 | 2 +- ...n_nulls-6-7eea211c80e7f1146098e80ffb890d67 | 2 +- ...n_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 | 2 +- ...n_nulls-8-609f6bd812a44b20da0a39c827e4d870 | 2 +- ...n_nulls-9-ef4b27a877efc377cd5750af6725194b | 2 +- ...llsafe-2-5bb63fafa390b1d4c20e225a8a648dcf} | 0 ...nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 | 0 ...lsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0} | 0 ...llsafe-20-d6fc260320c577eec9a5db0d4135d224 | 0 ...lsafe-21-24332b9390108fb3379e1acc599293a1} | 0 ...llsafe-21-a60dae725ffc543f805242611d99de4e | 0 ...llsafe-22-24c80d0f9e3d72c48d947770fa184985 | 0 ...lsafe-22-4be80634a6bd916e3ebd60a124f0a48e} | 0 ...llsafe-23-3fe6ae20cab3417759dcc654a3a26746 | 0 ...lsafe-23-e4425d56be43c21124d95160653ce0ac} | 0 ...join_rc-5-1aef75afe38d512addb44dbf9a650263 | 2 +- ...order2-11-f8460b061fa00f6afbfe8eeccf4d3564 | 2 +- ...eorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...order2-4-c0f14def6a135cc50cba364e810ce28e} | 0 ...eorder2-5-4abc4f450a58ccdd0df2e345f1276979 | 0 ...order2-5-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...eorder2-6-3fda17e4414d191f837631438a19e700 | 0 ...order2-6-8eb53fb8f05a43ee377aa1c927857e7c} | 0 ...eorder2-7-512b75ccb9459a6334da1d9699f4a5ec | 0 ...order2-7-5f4cfbbe53c5e808ee08b26514272034} | 0 ...eorder2-9-26ffed826eceda953b7124ee39ace828 | 2 +- ...order3-11-d6392b851f7dd5e3705e8ff51e1c6583 | 2 +- ...eorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...order3-4-c0f14def6a135cc50cba364e810ce28e} | 0 ...eorder3-5-4abc4f450a58ccdd0df2e345f1276979 | 0 ...order3-5-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...eorder3-6-3fda17e4414d191f837631438a19e700 | 0 ...order3-6-8eb53fb8f05a43ee377aa1c927857e7c} | 0 ...eorder3-7-512b75ccb9459a6334da1d9699f4a5ec | 0 ...order3-7-5f4cfbbe53c5e808ee08b26514272034} | 0 ...eorder3-9-9d45e66a3990b7c53fd780f07cd52e13 | 2 +- ...order4-10-1d6b948747ac31296710a491a5652e3f | 2 +- ...eorder4-12-203aed2c4298eba4a3c51816a21a1c1 | 2 +- ...eorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...order4-3-c0f14def6a135cc50cba364e810ce28e} | 0 ...eorder4-4-4abc4f450a58ccdd0df2e345f1276979 | 0 ...order4-4-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...eorder4-5-3fda17e4414d191f837631438a19e700 | 0 ...order4-5-8eb53fb8f05a43ee377aa1c927857e7c} | 0 ...eorder4-6-24ca942f094b14b92086305cc125e833 | 2 +- ...reorder4-8-4bbb8e937245e2fa7cafbb423814754 | 2 +- ...n_star-10-57ce75f989b3b3bfd2f2eceb228e892e | 0 ..._star-10-a9e579038e3d4826fdae475d7058ab82} | 0 ..._star-11-72730ecdad9c0fd4c6ce64a0cb89fb74} | 0 ...n_star-11-eba1397e66f25cba4fd264209cc92bae | 0 ...n_star-12-89b53ae954ec88171ef87e0459f6eb82 | 0 ..._star-12-f581d6d305d652cd0f4e4fa912eb578d} | 0 ...n_star-13-342b7249c9ce1484869169b1b33191cb | 0 ..._star-13-7268564732cbb7489248f9d818f80c14} | 0 ..._star-14-2ee0fcf000f8687fc8941bf212477e57} | 0 ...n_star-14-75513308d30b781fd2e06d81963c4363 | 0 ...n_star-15-24a0b1d0257bad4f85b3a122acf6bef9 | 0 ..._star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9} | 0 ...n_star-16-24ca942f094b14b92086305cc125e833 | 2 +- ...n_star-17-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...n_star-18-1c22e451845667bd6b4eac3c49c36965 | 2 +- ...n_star-20-76473cb68a69b9408207fd43ddae9339 | 2 +- ...n_star-22-429cfd140488242d569aee6594aae76b | 2 +- ...n_star-24-e11ab68d72d874c7c6c658c1018f5a49 | 2 +- ...n_star-26-a412a0620a359a62e2ab4e45fa8e0330 | 2 +- ...n_star-28-98a3f4d8a2209e771c57462d4b723ff9 | 2 +- ...in_star-8-a957982d8981ff0a35397ca449297024 | 0 ...n_star-8-c3d53a4daab9614a09870dc8e9571f74} | 0 ...n_star-9-3f7ccccc2488de5f33a38cb3cc3eb628} | 0 ...in_star-9-904e30d8615eb411fb24c2cc08df94f4 | 0 ..._thrift-0-811b62ecbacdb26c67fa260ff3747a41 | 6 - ..._thrift-1-4f47dd107d2be1212411bda214c0d1db | 0 ..._thrift-2-d4bfb7bab262dc2898431680711dec1b | 11 - ...join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 | 0 ...in_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 ...in_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 | 0 ...in_view-2-eda40dba9678df62dde73fc5dafb2b44 | 0 ...in_view-3-eccb00d8dada8ab56a48c373e381e02b | 0 ...in_view-5-85baeea910adc4589bc3ec2ce0b1e856 | 0 ...in_view-6-c37b7962ab7371f94a9c54d61f7638ef | 0 ...in_view-7-a14cfe3eff322066e61023ec06c7735d | 0 ...in_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 ...in_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 | 0 ...noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 | 0 ...noalias-1-6d5806dd1d2511911a5de1e205523f42 | 2 - ...noalias-2-155b3cc2f5054725a9c2acca3c38c00a | 0 ...noalias-3-3b7045ace234af8e5e86d8ac23ccee56 | 2 - ..._noalias-4-e1eca4e08216897d090259d4fd1e3fe | 0 ...noalias-5-16d227442dd775615c6ecfceedc6c612 | 0 ...w_outer-0-b66c363516d1f717765df9b91da3b5b4 | 0 ...w_outer-1-8d4332785ff69bb86607700c133d0baa | 10 - ...w_outer-2-b4474ec12d042fca7a21a2cd551c1068 | 0 ...w_outer-3-57b2ffd7b60708551238c491a2a8685d | 10 - ...w_outer-4-96fe3dc4f8116e535c322437a59b5e4e | 0 ...w_outer-5-2ec3aeb923621c64da042402709e64e9 | 0 ...w_outer-6-511e4df505342e04c20e50fda8962120 | 10 - ...ew_ppd-10-a537ad7282d1c9957cdae74ad87c790b | 6 + ...ew_ppd-9-dc6fea663d875b082d38bd326d21cd95} | 0 ...emijoin-5-9c307c0559d735960ce77efa95b2b17b | 0 ...mijoin-5-aba449db0d4fe6dc9771426e102bb543} | 0 ...emijoin-6-82921fc96eef547ec0f71027ee88298c | 0 ...mijoin-6-9f50dce576b019c0be997055b8876621} | 0 ...emijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 | 0 ...mijoin-7-fff6ca40e6048d52dc2d3afc68e8353e} | 0 ...oin_mr-1-5b2e555868faa404ea09928936178181} | 0 ...join_mr-1-aa3f07f028027ffd13ab5535dc821593 | 0 ...join_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 | 0 ...oin_mr-3-c019cb2a855138da0d0b1e5c67cd6354} | 0 ...gth.udf-0-e85ebb8ce5d939964fd87bd13b326c02 | 2 +- ...ushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb | 1 - ...ushdown-1-d0a93f40892e3894460553b443c77428 | 1 - ...shdown-10-19842c7708a0787e59cc5e6b8c3a4250 | 0 ...shdown-11-b435672262fc3c02d1ed1a93ff8d856f | 20 - ...shdown-12-a3516c212d6c79986536edbd9c961098 | 0 ...shdown-13-93906f7e87e5e3dc4b9590ec13b6af78 | 20 - ...shdown-14-cfae77212d164efb18208f71332fd368 | 0 ...shdown-15-62a90d3ba54cc68055b3502c3864a3c1 | 0 ...ushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 | 0 ...shdown-17-11c349c0db0f869be88351256650fe60 | 20 - ...shdown-18-66ecdd019c936dec1ab5c9389c9071b3 | 0 ...shdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 - ...ushdown-2-4d1e292b00635298240ff909be64dce4 | 0 ...shdown-20-d4de935f7a059cce44889c6ba69cfddb | 0 ...shdown-21-f04dee0f94443ca01320657897cbb914 | 20 - ...shdown-22-4d378725e22e7e48c861983ba935bf5e | 1 - ...shdown-23-53b4dd4899cb2ba681a801a3b5ca155a | 0 ...shdown-24-cb58113b9dfd3e93d4623cba6a090976 | 100 - ...shdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a | 0 ...shdown-26-e691b284f1b830d7b83a36750105737c | 100 - ...ushdown-3-cc674af3ce71c06670e366932df43462 | 20 - ...ushdown-4-81bbb7300da27bc63f7a139677faac3f | 0 ...ushdown-5-c0664bc5bff121823ac2f37df13d6bd7 | 20 - ...pushdown-6-b722630e977a5ca509234fc417d7b30 | 0 ...ushdown-7-e8c8087a3910ea53a725b01b9d05b921 | 20 - ...ushdown-8-6194a19c1d5065731ec26ea1d5a390e1 | 0 ...ushdown-9-1446f634128a1c3e7cd224ea82452d0a | 20 - ...egative-7-fb7bf3783d4fb43673a202c4111d9092 | 0 ..._double-0-10ef1098e35d900983be3814de8f974f | 0 ...double-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...double-1-10ef1098e35d900983be3814de8f974f} | 0 ..._double-1-3863c17e03c9c1cd68452106a8721d13 | 1 - ..._double-2-3863c17e03c9c1cd68452106a8721d13 | 1 + ...l_ints-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...al_ints-0-5ffd1b49cdda4149aef2c61c53a56890 | 0 ...l_ints-1-5ffd1b49cdda4149aef2c61c53a56890} | 0 ...al_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 | 1 - ...al_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 | 1 + ...string-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...l_string-0-9b48d41e5c9e41ddc070e2fd31ace15 | 0 ..._string-1-2cf4b7268b47246afdf6c792acca379d | 1 - ..._string-1-9b48d41e5c9e41ddc070e2fd31ace15} | 0 ..._string-2-2cf4b7268b47246afdf6c792acca379d | 1 + ...ry_data-0-491edd0c42ceb79e799ba50555bc8c15 | 0 ...ry_data-1-5d72f8449b69df3c08e3f444f09428bc | 0 ...ary_data-2-242b1655c7e7325ee9f26552ea8fc25 | 0 ...ry_data-3-2a72df8d3e398d0963ef91162ce7d268 | 0 ...ry_data-4-3ebc340f7f63740f8534706d42dd37ca | Bin 114 -> 0 bytes ..._part1-10-c66fea7ad025cd1f513a98a4cc1036b1 | 1000 +++++++ ..._part1-10-fca9513ea05bfb8b7e0e6f337d184d66 | 1000 ------- ..._part1-11-8b6be78ae0527e5b7efd7db758966853 | 1000 +++++++ ..._part1-11-9c82167763a771c175c656786d545798 | 1000 ------- ...n_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 | 14 +- ...n_part10-0-463330cf55370dbe92d6ed74ef91302 | 2 +- ..._part10-2-151ba0c3b8317902f1235ac07d58135e | 14 +- ..._part10-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ..._part10-4-a4fb8359a2179ec70777aad6366071b7 | 2 +- ..._part10-5-16367c381d4b189b3640c92511244bfe | 2 +- ..._part10-8-245027204484e281e1cfaf74386d2967 | 2 +- ..._part10-9-a646fd106fe73b8753fe3fee86d56ebf | 1000 +++++++ ..._part10-9-eb29a5b2392bf821b2ff51b70863d531 | 1000 ------- ...n_part11-0-463330cf55370dbe92d6ed74ef91302 | 2 +- ..._part11-2-4301f87a8dbf9576788637386e26f9a2 | 14 +- ..._part11-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ..._part11-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ..._part11-5-ea607fbed28d20e5726f4501285d698d | 2 +- ..._part11-6-a4fb8359a2179ec70777aad6366071b7 | 2 +- ..._part11-8-9a4433518ac9ff49cb4b71812705adbb | 2 +- ..._part11-9-3889a0cba1cf3c8f8c2e67957e69406a | 2 +- ...n_part12-0-463330cf55370dbe92d6ed74ef91302 | 2 +- ..._part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b | 14 +- ..._part12-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ..._part12-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ..._part12-5-a4fb8359a2179ec70777aad6366071b7 | 2 +- ..._part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 | 2 +- ...n_part12-8-fd656b581b8f8fbb8ac22f444dbc345 | 2 +- ...n_part13-0-463330cf55370dbe92d6ed74ef91302 | 2 +- ..._part13-2-d52536b8ac62f6e8152e394fe135a3e0 | 14 +- ..._part13-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ..._part13-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ..._part13-5-a4fb8359a2179ec70777aad6366071b7 | 2 +- ...n_part13-8-930d8e7a94f0cdf922322dae4a02e16 | 2 +- ..._part13-9-c8de411bc094b37b59a2eb0baf6de55d | 2 +- ..._part14-1-253e2a15bfaef9aa781dc29fa324b51e | 8 +- ..._part14-2-a4fb8359a2179ec70777aad6366071b7 | 2 +- ..._part14-3-16367c381d4b189b3640c92511244bfe | 2 +- ..._part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 | 2 +- ..._part14-7-7c931249465f330d51ef0610f214429e | 2 +- ...t14_win-1-253e2a15bfaef9aa781dc29fa324b51e | 8 +- ...t14_win-2-a4fb8359a2179ec70777aad6366071b7 | 2 +- ...t14_win-3-16367c381d4b189b3640c92511244bfe | 2 +- ...t14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 | 2 +- ...t14_win-7-7c931249465f330d51ef0610f214429e | 2 +- ..._part15-2-1379abc2de057dc6d240a526f0dd8a3c | 0 ..._part15-3-9940fad8d8590e60bd726e9503ae4fa9 | 0 ..._part15-4-fa01199bab3e6aa47a82c9aec88aa76a | 0 ..._part15-5-9940fad8d8590e60bd726e9503ae4fa9 | 3 - ...n_part2-1-845923af04bb331c9f5995a3a3e84424 | 14 +- ...n_part2-2-dc129f70e75cd575ce8c0de288884523 | 2 +- ...n_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...n_part2-4-a4fb8359a2179ec70777aad6366071b7 | 2 +- ...n_part2-7-86ffa99b03fa88235b61bf1af7062c33 | 2 +- ...n_part2-8-a1ff8a12d94378e7e1165bd78cf724cf | 2 +- ...n_part2-9-3f29de9877006f9448272ef2422d6132 | 2 +- ...n_part3-2-dbbba335c008a61a13c1472b34470397 | 14 +- ...n_part3-8-c32770da4784bfea4d0dd77fdcba4f0a | 2000 +++++++++++++ ...n_part3-8-c3b6f0cc860b84851dd60c53b36a6437 | 2000 ------------- ..._part4-10-297cf42ec45b5aa78b80d9c35a79555a | 1500 ++++++++++ ..._part4-10-7f0c1e54518d4d1d345577a100e670e8 | 1500 ---------- ..._part4-11-1313644c9dad948bfcebd7386c309ab7 | 2500 ----------------- ..._part4-11-24618a43c4656b72f81683b45708045e | 2500 +++++++++++++++++ ...n_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd | 14 +- ...n_part5-1-9a4d1f6a14227bb66bd01557a464da8b | 8 +- ...n_part6-2-779aa345cf2875089312ec26b046415d | 14 +- ...n_part7-2-5d4c3c48f53d55e26ca142ee70d1706a | 14 +- ...n_part8-2-9e1df41acabef83f62464f52c2396c8a | 14 +- ...n_part8-9-704bd110d9aaa2ac678b7fbf645abdb9 | 2000 +++++++++++++ ...n_part8-9-a7456fceb35f62a282db750384f480db | 2000 ------------- ...n_part9-2-748ac33315295f8e55f2480f7714c27a | 14 +- ...yn_part9-9-6954f5fc8dd82ca2c076ab8bcdbc148 | 1000 +++++++ ...n_part9-9-72b1ad2231269b704130903b35ac04bb | 1000 ------- ...success-0-84028c4ca541d126baffc20d6d876810 | 0 ...success-1-c84cd1b5b491bded8ac3b0521de599c5 | 0 ...success-3-9ee887603dcba5200918ae5200afa5d5 | 0 ...he_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f | 0 ...e_name-1-d19201e2fcaee4d451292bd740e6c637} | 0 ...e_name-2-ad8795e50f5998ea1d2eb64a0c02e6e5} | 0 ...oad_fs2-0-517732da2c84ae17095b0e1d96f74d97 | 0 ...oad_fs2-1-5018c84e09be70bf663594a89f3ad731 | 0 ...oad_fs2-2-94d2317b453b3b49bb9f2b58040b4748 | 0 ...he_name-0-f99b4f29506d65c841fb1db18263fbcc | 1 - ...he_name-1-b64a19f7101a4fb3b5d08b2f6e296400 | 1 - ...he_name-2-2087e00fe000e00f64e819dca59be450 | 0 ...success-0-fa705a031ff5d97558f29c2b5b9de282 | 0 ...success-2-9ee887603dcba5200918ae5200afa5d5 | 0 ...success-0-84028c4ca541d126baffc20d6d876810 | 0 ...success-2-9ee887603dcba5200918ae5200afa5d5 | 0 ...adpart1-1-4bf1504274319c44d370b58092fe016c | 0 ...dpart1-1-6cc94d19c536a996592629f7c82c2ac9} | 0 ...adpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 | 2 +- ...adpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 | 2 +- ...art_err-2-21fe8ff9059167209647e7ea086f483e | 0 .../lock1-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../lock1-1-3e95421993ab28d18245ec2340f580a3 | 0 .../lock1-2-c0c18ac884677231a41eea8d980d0451 | 0 .../lock2-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../lock2-1-3e95421993ab28d18245ec2340f580a3 | 0 .../lock2-2-c0c18ac884677231a41eea8d980d0451 | 0 .../lock2-3-27ad2962fed131f51ba802596ba37278 | 0 .../lock2-4-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock2-5-e8d1d10c308a73eef78dde414a5e40ca | 0 .../lock3-0-27ad2962fed131f51ba802596ba37278 | 0 .../lock3-1-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock3-2-b1ca816784e88f105b2fce1175340c33 | 0 .../lock3-5-8096935c5c1755f9b88583e8c72921ac | 0 .../lock3-6-598ba296ba1c6931f4161a9f50b00cbe | 0 .../lock4-1-27ad2962fed131f51ba802596ba37278 | 0 .../lock4-2-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock4-3-b1ca816784e88f105b2fce1175340c33 | 0 .../lock4-6-8096935c5c1755f9b88583e8c72921ac | 0 .../lock4-7-598ba296ba1c6931f4161a9f50b00cbe | 0 ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...oin_ppr-2-3d41e966f69a64babb783d1aad0f1b73 | 2 +- ...oin_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef | 2 +- ...join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 | 2 +- ...join_ppr-8-6fca189c46645f124d5fcb82564b703 | 2 +- ...apjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d | 1 - ...apjoin1-1-abd9364d276ec89352232da5e2237768 | 0 ...pjoin1-10-c08fefa00b89f50dd365208151593505 | 10 - ...pjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 | 0 ...pjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 | 10 - ...apjoin1-2-fe84593f006c85e68fbb797394cdccd0 | 10 - ...apjoin1-3-8439a0592619790b64d16d2506f2233d | 0 ...apjoin1-4-c08fefa00b89f50dd365208151593505 | 10 - ...apjoin1-5-72068bd4cdac40e4d18fd729f39855ba | 0 ...apjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 | 10 - ...apjoin1-7-757dfb540b8a49b3663f8caba7476ec5 | 0 ...apjoin1-8-fe84593f006c85e68fbb797394cdccd0 | 10 - ...apjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 | 0 ...istinct-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...istinct-1-a00d1791b7fa7ac5a0505d95c3d12257 | 2 +- ...stinct-11-4489654b888efc588b13ee1cda1b6a9f | 2 +- ...stinct-12-1d351f7e821fcaf66c6f7503e42fb291 | 2 +- ...stinct-13-a7dc16cb82c595b18d4258a38a304b1e | 2 +- ...stinct-15-4489654b888efc588b13ee1cda1b6a9f | 2 +- ...istinct-3-4489654b888efc588b13ee1cda1b6a9f | 2 +- ...istinct-4-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...istinct-5-a7dc16cb82c595b18d4258a38a304b1e | 2 +- ...istinct-7-4489654b888efc588b13ee1cda1b6a9f | 2 +- ...istinct-8-1d351f7e821fcaf66c6f7503e42fb291 | 2 +- ...istinct-9-a00d1791b7fa7ac5a0505d95c3d12257 | 2 +- ...mapjoin-0-24ca942f094b14b92086305cc125e833 | 2 +- ...mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...mapjoin-2-25fc734982956a164adde6bb1d4d8751 | 2 +- ...apjoin-4-5166a5b9d30dfacbe33dd909c0df6310} | 0 ...mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 | 0 ...mapjoin-5-5ee898cab883074f3297198c52445ee4 | 20 + ...mapjoin-5-c47698bac140454637a999e583941ce7 | 2 - ...mapjoin-6-dca8c08a235b45d1cdcb94e363afb17} | 0 ...mapjoin-7-fddbdea343a9ddb5f8dedc18147640b7 | 2 + ...apjoin-8-2be637ed4f6146e8525ae1a863e72736} | 0 ...mapjoin-9-c47698bac140454637a999e583941ce7 | 2 + ...ubquery-0-24ca942f094b14b92086305cc125e833 | 2 +- ...ubquery-1-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...ubquery-2-25fc734982956a164adde6bb1d4d8751 | 2 +- ...ubquery-4-7df121f9774cb23edc557b98ad1e1924 | 2 +- ...ubquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b | 2 +- ...query2-10-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...query2-11-25fc734982956a164adde6bb1d4d8751 | 2 +- ...query2-13-c876a518451059f17fc15e29f6f57951 | 2 +- ...bquery2-6-5353ee601eb42d5842690d3941683be1 | 0 ...query2-6-9bf06af695892b0d7067d5b30e0b2425} | 0 ...query2-7-c6b0cdb137f13f8362c0c49c544151a4} | 0 ...bquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 | 0 ...query2-8-131ae5ecfff2733b04bdfada0108cf40} | 0 ...bquery2-8-d524906728fef9f559709fe0922ab24e | 0 ...bquery2-9-24ca942f094b14b92086305cc125e833 | 2 +- ...t_outer-0-407016bf2679fb9e9d076a2d115e859d | 2 +- ..._outer-10-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 | 2 +- ..._outer-12-80993ab7f757001e1f058bf8609f8420 | 2 +- ..._outer-14-7fe52008c4a98853d086d17fc3c21906 | 2 +- ...t_outer-6-7fe52008c4a98853d086d17fc3c21906 | 2 +- ...t_outer-8-dfb08d397d3fe163d75c3b758097b68a | 2 +- ...t_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 | 2 +- ...reduce2-3-adea843673e541da8a735a5a34e7c7dc | 2 +- ...reduce3-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../merge1-0-593999fae618b6b38322bc9ae4e0c027 | 2 +- ...merge1-12-2e8e4adbfb21f25e7557dd86363c7138 | 2 +- .../merge1-4-3277fe538b66923cd879b45371838d2b | 2 +- .../merge2-0-b12e5c70d6d29757471b900b6160fa8a | 2 +- .../merge2-1-593999fae618b6b38322bc9ae4e0c027 | 2 +- ...merge2-17-2e8e4adbfb21f25e7557dd86363c7138 | 2 +- .../merge2-2-c95dc367df88c9e5cf77157f29ba2daf | 2 +- .../merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 | 2 +- .../merge2-4-84967075baa3e56fff2a23f8ab9ba076 | 2 +- .../merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea | 2 +- .../merge2-9-b81efaa65e1263e48278ef9062cca1dd | 2 +- ...merge4-10-692a197bd688b48f762e72978f54aa32 | 0 ...merge4-11-ca0336ac3f600cb8b4230d9904686868 | 1500 ---------- ...merge4-12-62541540a18d68a3cb8497a741061d11 | 0 ...merge4-13-ed1103f06609365b40e78d13c654cc71 | 0 ...merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 | 3 - ...merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a | 1 - .../merge4-5-3d24d877366c42030f6d9a596665720d | 0 .../merge4-6-b3a76420183795720ab3a384046e5af | 0 .../merge4-7-631a45828eae3f5f562d992efe4cd56d | 0 .../merge4-8-ca0336ac3f600cb8b4230d9904686868 | 1000 ------- .../merge4-9-ad3dc168c8b6f048717e39ab16b0a319 | 0 ...rtition-0-a4fb8359a2179ec70777aad6366071b7 | 1 - ...rtition-1-16367c381d4b189b3640c92511244bfe | 1 - ...tition-10-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...tition-11-a49c9ee01ce8858a5f00c05523329200 | 1 - ...tition-12-dbf4eae8430a97a6e70b1c6222218cd3 | 0 ...tition-13-16adcdb0e324ad233769e124b5b349da | 0 ...tition-14-79da9a1ce5c2d058b924387ac9fcde92 | 500 ---- ...tition-15-d60297fed03b455c29daa4afb4d1e858 | 14 - ...tition-16-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...tition-17-b12e5c70d6d29757471b900b6160fa8a | 1 - ...tition-18-593999fae618b6b38322bc9ae4e0c027 | 1 - ...tition-19-a49c9ee01ce8858a5f00c05523329200 | 1 - ...rtition-2-190cefc93e46906e404039de0fd5f513 | 0 ...tition-20-d295db835d4fdeea34298702295ff7c5 | 0 ...tition-21-9b9493a68ef7663e95ad86d02c45ec88 | 0 ...tition-22-79da9a1ce5c2d058b924387ac9fcde92 | 500 ---- ...tition-23-d60297fed03b455c29daa4afb4d1e858 | 14 - ...tition-24-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...tition-25-b12e5c70d6d29757471b900b6160fa8a | 1 - ...tition-26-593999fae618b6b38322bc9ae4e0c027 | 1 - ...tition-27-a49c9ee01ce8858a5f00c05523329200 | 1 - ...tition-28-ef7b35be7210f099d46448994d9dc605 | 0 ...tition-29-876c8fa15a32ac5bfcc6cb60993f6a4d | 0 ...rtition-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition-30-79da9a1ce5c2d058b924387ac9fcde92 | 500 ---- ...tition-31-d60297fed03b455c29daa4afb4d1e858 | 14 - ...rtition-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...rtition-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...rtition-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...rtition-7-777de794b7f27ea63f29a9784663393b | 0 ...rtition-8-2f1578dbc029b62daa9d47d8fa473960 | 1 - ...rtition-9-dc129f70e75cd575ce8c0de288884523 | 1 - ...tition2-0-a4fb8359a2179ec70777aad6366071b7 | 1 - ...tition2-1-16367c381d4b189b3640c92511244bfe | 1 - ...ition2-10-43d53504df013e6b35f81811138a167a | 1 - ...ition2-11-b12e5c70d6d29757471b900b6160fa8a | 1 - ...ition2-12-593999fae618b6b38322bc9ae4e0c027 | 1 - ...ition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 | 1 - ...tition2-14-3a4c24fd561f459025264baa3fb6d87 | 1 - ...ition2-15-dbf4eae8430a97a6e70b1c6222218cd3 | 0 ...ition2-16-16adcdb0e324ad233769e124b5b349da | 0 ...ition2-17-d60297fed03b455c29daa4afb4d1e858 | 14 - ...tition2-2-190cefc93e46906e404039de0fd5f513 | 0 ...tition2-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition2-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...tition2-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...tition2-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...tition2-7-777de794b7f27ea63f29a9784663393b | 0 ...tition2-8-6f7f59de1fbd607e844a2dc9394a2df8 | 0 ...tition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba | 0 ...tition3-0-a4fb8359a2179ec70777aad6366071b7 | 1 - ...tition3-1-16367c381d4b189b3640c92511244bfe | 1 - ...ition3-10-d176a1b243ac7190fbc319d73a164e2d | 0 ...ition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 | 0 ...ition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a | 0 ...ition3-13-c512eee654e7313ff9c6efb35c5b0a88 | 0 ...ition3-14-ea3e89ffe987e20dffd8388a2dbcc260 | 0 ...ition3-15-e525a096de36a3d157db1b4947e1fbb0 | 0 ...ition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 | 4 - ...ition3-17-43d53504df013e6b35f81811138a167a | 1 - ...ition3-18-b12e5c70d6d29757471b900b6160fa8a | 1 - ...ition3-19-593999fae618b6b38322bc9ae4e0c027 | 1 - ...tition3-2-190cefc93e46906e404039de0fd5f513 | 0 ...ition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 | 1 - ...tition3-21-3a4c24fd561f459025264baa3fb6d87 | 1 - ...ition3-22-bf8877c86d4b4cd7da2939cdf517acc5 | 0 ...ition3-23-3ffba3098571099bc2b13614ae3defc5 | 0 ...ition3-24-c9f91a11d9be1e981e6ec284572bbb2a | 4 - ...ition3-25-d60297fed03b455c29daa4afb4d1e858 | 14 - ...tition3-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition3-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...tition3-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...tition3-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...tition3-7-777de794b7f27ea63f29a9784663393b | 0 ...tition3-8-6916eceaa04091d1453a7d0d5257213c | 0 ...tition3-9-8d0305d089aa5198601cc39073fff528 | 0 ...s_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 | 0 ...s_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 | 0 ..._mixed-10-abb02d2cadc535ff51660d83e6609dc8 | 0 ..._mixed-11-725ed77dd110398f461926f190b4b5c8 | 0 ..._mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b | 0 ...s_mixed-2-de24edb80e51049d241fa0ce2e3165ff | 0 ...s_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 | 0 ...s_mixed-4-24a8048db8d561e28bcb4941498b9687 | 0 ...s_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 | 0 ...s_mixed-6-e108b1560a601946194cecaf4da12491 | 0 ...ns_mixed-7-d2068e6569b5d253932ce9d59be5221 | 0 ...s_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 | 0 ...s_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 | 0 ...taonly1-0-f0c7b56d5ad529ae6c98875501a9548d | 0 ...taonly1-1-4d93504b19d34fd360de6af7fb1f3715 | 0 ...aonly1-10-1c7bb3877b2e1e7bb9166537715d442d | 0 ...aonly1-11-30df79b460916f3f67ccf1d7b7a076f2 | 0 ...taonly1-3-7980a98d580a002b7ad7eef780039f67 | 0 ...taonly1-4-4d93504b19d34fd360de6af7fb1f3715 | 0 ...taonly1-6-537256f669bc9101d4834df67aae8cdf | 0 ...taonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 | 0 .../mi-0-a4fb8359a2179ec70777aad6366071b7 | 1 - .../mi-1-16367c381d4b189b3640c92511244bfe | 1 - .../mi-2-abf8847fb25b96e0f9477808d8378e5e | 0 .../mi-3-b66a495f7bdf106a7886b72267b8659d | 0 .../mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c | 0 .../mi-5-4aad9be12cbe4e7be5540924e6b5e7dc | 1 - .../mi-6-b331d7aa963adac3b2e1de803f90e199 | 1 - .../mi-7-fca4e06de103c3cbb675fa43e7077800 | 500 ---- .../mi-8-e946bdb81b0a831908c1c8196fdff215 | 309 -- ...pJoin1-10-24ca942f094b14b92086305cc125e833 | 2 +- ...pJoin1-12-204073e1287b0582d50e652d466f1e66 | 2 +- ...pJoin1-13-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...pJoin1-14-25fc734982956a164adde6bb1d4d8751 | 2 +- ...pJoin1-16-c14b300770b329ecb71e0275c88532d3 | 2 +- ...pJoin1-18-49bc7f430b2591978067ca8f7d181cee | 2 +- ...apJoin1-22-25e434b6d05e08fdd5f4d9957438917 | 2 +- ...pJoin1-24-feed626e3216bcbda66b17f48305b5a1 | 2 +- ...pJoin1-25-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...pJoin1-26-25fc734982956a164adde6bb1d4d8751 | 2 +- ...pJoin1-28-feed626e3216bcbda66b17f48305b5a1 | 2 +- ...pJoin1-29-ea23403b9eb55e8b06d1c198e439569f | 2 +- ...pJoin1-31-feed626e3216bcbda66b17f48305b5a1 | 2 +- ...pJoin1-32-e93301ee4ba157b466d7460775f3d350 | 2 +- ...pJoin1-34-feed626e3216bcbda66b17f48305b5a1 | 2 +- ...apJoin2-0-24ca942f094b14b92086305cc125e833 | 2 +- ...apJoin2-1-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...pJoin2-10-1905c7759350b107679aef86226739f8 | 2 +- ...pJoin2-11-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...pJoin2-13-6b984427a771fe650fa875be98722cbe | 2 +- ...pJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...pJoin2-16-6b984427a771fe650fa875be98722cbe | 2 +- ...pJoin2-17-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...pJoin2-19-77324702b091d514ca16d029f65d3d56 | 2 +- ...apJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 | 2 +- ...pJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...pJoin2-22-77324702b091d514ca16d029f65d3d56 | 2 +- ...pJoin2-27-d28d0f671f5d913a56d75812d24cca8e | 2 +- ...Join2-28-c14792ccac2ca64e3e9e21af4fd12d2c} | 0 ...Join2-29-b9d963d24994c47c3776dda6f7d3881f} | 0 ...Join2-30-6d1c7f7014fc709148b0f401c5f23cb3} | 0 ...pJoin2-31-2e9c4d5e2bd709e96f311488ada116b0 | 1000 +++++++ ...apJoin2-4-5ede8243cc4ba2fbd24a77578502a656 | 2 +- ...apJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 | 2 +- ...apJoin2-7-5ede8243cc4ba2fbd24a77578502a656 | 2 +- ...apJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 | 2 +- ...ert_gby-4-521e0c1054cfa35116c02245874a4e69 | 2 +- ...ert_gby-8-521e0c1054cfa35116c02245874a4e69 | 2 +- ...ert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...rt_gby2-0-b3ee4be40513342084411c5333416d69 | 0 ...rt_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 | 0 ...rt_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 | 0 ...rt_gby2-3-538a02b95c066b307652c8d503470c8e | 0 ...rt_gby2-4-521e0c1054cfa35116c02245874a4e69 | 1 - ...rt_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 | 1 - ...t_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...rt_gby3-6-521e0c1054cfa35116c02245874a4e69 | 2 +- ...rt_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...rt_gby3-9-521e0c1054cfa35116c02245874a4e69 | 2 +- ...n_union-0-24ca942f094b14b92086305cc125e833 | 2 +- ..._complex-0-6a7c4841dab05ebae84309c9571bec6 | 0 ...complex-1-abac744dee1a4f4152781b1565fe1364 | 5 - ...complex-2-f07e9814229ed440bd0ac7e4bd924ce9 | 8 - ...complex-3-fb014bc1e9bfd919a3703cf5afb77448 | 0 ...complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d | 2 - ...virtual-0-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...virtual-1-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...irtual-10-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...irtual-11-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...irtual-12-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...irtual-13-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...irtual-14-67b834deba21676e02c155b25195a019 | 0 ...irtual-15-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...irtual-16-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...irtual-17-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...virtual-2-67b834deba21676e02c155b25195a019 | 0 ...virtual-3-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...virtual-4-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...virtual-5-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...virtual-6-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...virtual-7-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...virtual-8-67b834deba21676e02c155b25195a019 | 0 ...virtual-9-52f90e6bf3f2c17b82ed34318d2c612f | 0 .../newline-2-4eb54a664e549614d56ca088c8867d | 0 ...s_subq1-1-f91e87acd51f7477d96620b5f9deece6 | 2 +- ...plicate-1-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...plicate-3-a873d2d6991308b21ecdc46ac777c716 | 2 +- ...plicate-4-24ca942f094b14b92086305cc125e833 | 2 +- ...plicate-6-a873d2d6991308b21ecdc46ac777c716 | 2 +- ...input37-0-6ed1b2ff177492c003161ee91e982c10 | 0 ...input37-1-e3ab2563222fb8678c7c269e09e1e8d8 | 0 ...t_into1-0-d3afbe52e3a1eb246a79c1320f82c480 | 0 ...t_into1-1-5c7fc72601c9add7bc86df7e4c24af63 | 0 ..._into1-10-3c8f9dff0a12ca2b929d04b4873a4681 | 0 ..._into1-11-6f59e35684a552a855e4dc3aee667092 | 1 - ..._into1-12-d3afbe52e3a1eb246a79c1320f82c480 | 0 ...t_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 | 0 ...t_into1-3-2983d09b973ea94bc701970a17fc3687 | 0 ...t_into1-4-a2f6c6c77e94833197095dc48643f9c9 | 1 - ...t_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 | 0 ...t_into1-6-2983d09b973ea94bc701970a17fc3687 | 0 ...t_into1-7-fe194a16b48b763e6efdf6fcc6116296 | 1 - ...t_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad | 1 - ...t_into1-9-461110270cec5b6d012b2c685cf2cce9 | 0 ..._alias1-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._alias2-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ll_cast-0-48a80d4fdc8009234af31ebcb6e03364 | 0 ...ll_cast-1-7257e6f8170e545962d27741353f672c | 1 - ...llgroup-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...llgroup-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...lgroup-11-54a5fd76cdeff565c8c7724695aca302 | 2 +- ...lgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...lgroup-13-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...lgroup-15-54a5fd76cdeff565c8c7724695aca302 | 2 +- ...llgroup-3-54a5fd76cdeff565c8c7724695aca302 | 2 +- ...llgroup-4-dbcec232623048c7748b708123e18bf0 | 2 +- ...llgroup-5-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...llgroup-7-54a5fd76cdeff565c8c7724695aca302 | 2 +- ...llgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...llgroup-9-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...lgroup2-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...lgroup2-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...group2-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...group2-13-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...lgroup2-4-dbcec232623048c7748b708123e18bf0 | 2 +- ...lgroup2-5-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...lgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...lgroup2-9-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...group3-1-61ead7f73d525e0d9e21beba91a3d39e} | 0 ...lgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 | 0 ...roup3-12-61ead7f73d525e0d9e21beba91a3d39e} | 0 ...group3-12-a4d6d87d12f0e353d7a3ae5c087dab44 | 0 ...roup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06} | 0 ...group3-13-f529994bef750d8edd02ede8a4323afb | 0 ...group3-15-cd90e16da918bac569e9f04faaedd280 | 2 +- ...roup3-18-113e2b587784d54d2a5b5f091389397e} | 0 ...group3-18-a24e178327b39f57ca7dfa2e69296bb2 | 0 ...roup3-19-77de4b2d65eee228848625cdd422178d} | 0 ...group3-19-b8f60d35c1c09c57efd4c339799e5404 | 0 ...group3-2-a5e12cfbc1799ce9fa9628d81b8c0b06} | 0 ...lgroup3-2-f529994bef750d8edd02ede8a4323afb | 0 ...group3-21-e09c6bf2f6112981793fbd4386d43ff6 | 2 +- ...lgroup3-4-cd90e16da918bac569e9f04faaedd280 | 2 +- ...group3-6-113e2b587784d54d2a5b5f091389397e} | 0 ...lgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 | 0 ...group3-7-77de4b2d65eee228848625cdd422178d} | 0 ...lgroup3-7-b8f60d35c1c09c57efd4c339799e5404 | 0 ...lgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 | 2 +- ...group5-1-642e12a05bf01a6716bfa15ed0012629} | 0 ...lgroup5-1-c75bafea030f127bce9a4a5dbb14f40b | 0 ...group5-3-35517978e09aa1bd1d15f51d11e08fd5} | 0 ...lgroup5-3-4492a9ce0d8502584b872860d53c449c | 0 ...lgroup5-5-60d7180632a63c79eeba47e30b854f4c | 2 +- ...linput2-2-21058230c4992a682c4adef9881fa9a2 | 2 +- ...lscript-1-3729d183a27e89a87ca9b9c9946e13a5 | 0 ...script-1-f07dfd6ef687e038083deca5941d8174} | 0 ...lscript-2-17238164053203d56d30704e2c098e80 | 0 ...script-2-bb1abcf2f4a2a5cd5c058104901627bb} | 0 ...lscript-4-472199a0c6b8f760a90863deb69e9710 | 2 +- ...pe_conv-0-c3f17f31d887479896bf454a2f6b15dc | 0 ...pe_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 | 1 - ..._create-0-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ..._create-1-b5209e6f850fc958dc9ebced21519684 | 0 ...create-10-3027edcda6a6030c6bff93681529c34d | 0 ...create-11-14ea7dcc6898979aaa61650e3fc46187 | 32 - ...create-12-dd590639ea8d5b27ccf3312be2562fc2 | 0 ...create-13-14ea7dcc6898979aaa61650e3fc46187 | 34 - ...create-14-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...create-16-3027edcda6a6030c6bff93681529c34d | 0 ...create-18-14ea7dcc6898979aaa61650e3fc46187 | 32 - ...create-19-a85f67347e5a19fc879ae83aa24c0ce4 | 0 ..._create-2-f375f322df98df65e6b1bd0d4ce8e208 | 0 ...create-20-176d469a0edba57404416535c7d48023 | 29 - ...create-21-e3accaf8ebc11bac7091db28d01fd2a7 | 0 ...create-22-31944ad765bbf262b4ebafdb06df59a6 | 3 - ...create-23-be779533ea8967231e644209114c8350 | 0 ...create-24-fe59ff341395bc347dfb9cfdee397da2 | 3 - ...create-25-c55e620d82921c40ebcdb94454ac1ead | 3 - ...create-26-aa4e0c8a3de340c1133d9fdddda8a18c | 3 - ..._create-27-2eefc0bb4283fc255409a5c41203c89 | 3 - ...create-28-cf8aa1014707dfe576820041e47436e2 | 3 - ...create-29-cee6b57822d79ce80e14e58663bf7b86 | 0 ..._create-3-8480c37b4f3a6768f459361cf8470dae | 0 ...create-30-b5e831bf4eb811ac4f06e9ac11a5283d | 0 ...create-31-c934a1ca351b76c5c4ef87cb5655f1d3 | 0 ...create-33-2151d07a07ab6b28782b7e8e4f848e36 | 0 ...create-36-ca15a32658195ecaffe11d11f487fb0a | 10 - ...create-37-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...create-38-b5209e6f850fc958dc9ebced21519684 | 0 ...create-39-f375f322df98df65e6b1bd0d4ce8e208 | 0 ..._create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 | 0 ...create-40-8480c37b4f3a6768f459361cf8470dae | 0 ...create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 | 0 ..._create-5-4f8e2d1f18399679a97c49631c4ef921 | 0 ..._create-6-2ae0fd655aa777b41015e4125f680324 | 32 - ..._create-7-27aa4a8093e80a7437266f349ea927c0 | 0 ..._create-8-14ea7dcc6898979aaa61650e3fc46187 | 32 - ..._create-9-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...reshold-0-a399c0ef0f1578f9d2456727008dee16 | 2 +- ...hreshold-3-1a7f96f49e2992b93d97edcbdb0075d | 2 +- ...reshold-6-272f5d299289829dc22cc31f70115dd9 | 0 ...eshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3} | 0 ...reshold-7-acf39b28d4b76382acf5c56c21932ff9 | 2 +- ...reshold-9-f7b722063a6948d22aaaab0707cddde1 | 2 +- ...rt_cols-0-bfdd54175515a0557f8bd427ec23c453 | 0 ...rt_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...rt_cols-2-ab12575b9b663420d9b3702f5a32a086 | 0 ...rt_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 | 0 ...rt_cols-4-1c1eab8fc34159875afe38eb2413434e | 0 ...rt_cols-5-c0dce07a38f3c540c2b17db0a495a657 | 10 - ..._strings-0-a338239d2fc4e22e23ab82fa5f62139 | 0 ...strings-1-a67f0c90e3f99257a7364cc96e5405cf | 0 ...strings-2-a34bd419a63852c1c75f195a495ff333 | 9 - ...strings-3-3339ace17de3201296847caf29c42e99 | 0 ...strings-4-eee6b7f3a881c8f56510bbd53aeebe1e | 10 - ...th_nulls-0-a338239d2fc4e22e23ab82fa5f62139 | 0 ...h_nulls-1-2c16215823a5e6904059a48a3077da4e | 0 ...h_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 | 0 ...h_nulls-3-bbe8d353c397b795e2732bd59648d291 | 0 ...h_nulls-4-a0231a52d63534679bfab3f6a0b8effc | 0 ...h_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f | 5 - .../order-1-57d93bd7619dfc460dfa763c12142bb9 | 2 +- .../order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 | 2 +- .../order2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- .../order2-2-4162aa366dc0836eed15cc819226907f | 2 +- ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...oin_ppr-2-b60ab17f7372863504804717c4276595 | 2 +- ...join_ppr-4-be64a45a645ea5363109323a378d335 | 2 +- ...arallel-0-23a4feaede17467a8cc26e4d86ec30f9 | 2 +- ...arallel-1-851e262128626126ae1ad87869db7c54 | 2 +- ...rallel-11-6230286bc168af7b010968b543690a2a | 2 +- ...rallel-12-73a915d42e62c0e895a82602a502ee43 | 2 +- ...arallel-2-43d53504df013e6b35f81811138a167a | 2 +- ...arallel-7-6230286bc168af7b010968b543690a2a | 2 +- ...arallel-8-73a915d42e62c0e895a82602a502ee43 | 2 +- ...arallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c | 2 +- ...orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d | 0 ...orderby-1-be7423a5e77b7289153f74bc3fd97f44 | 0 ...rderby-11-39767502cbda75590c0c4b8fd089b793 | 0 ...rderby-12-7943485bcc72b2040c45c62e45ac8853 | 0 ...rderby-13-988591bf9ab008fdc4d71560aa57a736 | 32 - ...rderby-14-d93951df2ffc18dc09ab0ba2d46f1823 | 48 - ...orderby-2-be7423a5e77b7289153f74bc3fd97f44 | 0 ...orderby-6-670ec1b1f28d92d72a924c29d622aa8f | 0 ...orderby-7-7943485bcc72b2040c45c62e45ac8853 | 0 ...orderby-8-988591bf9ab008fdc4d71560aa57a736 | 32 - ...orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 | 48 - ...star_by-0-57a4ea931689f9475b687292f34abfa4 | 2 +- ...star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 | 2 +- ...star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc | 2 +- ...star_by-3-498e2973594ccf45448ba19552bfb1cd | 2 +- ...star_by-4-dc9d9d8bbb6259c89a97065902a345ec | 2 +- ...star_by-5-6888c7f7894910538d82eefa23443189 | 2 +- ...star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 | 2 +- ...star_by-7-f36c71c612ab8da4f4191980c1b61fea | 2 +- ...artInit-0-aaa3124841a8140a41d3556a4ccaa6a5 | 0 ...artInit-1-c0ec92801bec7ece0a156d407b601f7b | 1 - ...artInit-2-5db6fb2267a648ac57af8f56f91cf9a2 | 1 - ...artInit-3-878a82ca7519e3eafd3f2661b2ac1b88 | 0 ...artInit-4-5db6fb2267a648ac57af8f56f91cf9a2 | 1 - ...l_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 | 2 +- ...l_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 12 +- ...l_props-4-c04c695a6ebed215889ce75edcb33eb4 | 2 +- ...s_empty-0-c367ba7f534037ab96efc7f2273508c7 | 2 +- ...s_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 12 +- ...th_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 | 2 +- ...th_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 12 +- ...th_star-4-c04c695a6ebed215889ce75edcb33eb4 | 2 +- ...rtcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 | 0 ...tcols1-1-b562ff3e36de23a686d122967a1f91c8} | 0 ...n_date-1-6b0952309c3ebdd6dcb7066891d1bd74} | 0 ...n_date-10-aad6078a09b7bd8f5141437e86bb229f | 1 + ..._date-11-a01c3791f59e819c750e213b6c65d084} | 0 ...n_date-12-2662a237c86cf9e4a150a4f1856b8239 | 1 + ...n_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 | 1 + ..._date-14-3f187dad9a2fdfc6f7a3566309016f9c} | 0 ..._date-15-e4366325f3a0c4a8e92be59f4de73fce} | 0 ...n_date-16-32cf81c1a36451eccb07b20dffd930ac | 1 + ...n_date-17-8654528691598a5eef8e3c1059d24117 | 1 + ...n_date-18-a1e769630ac3caed7325a3a256713b24 | 1 + ...n_date-19-95f1399e12124840caee7492c0f3036d | 1 + ...n_date-2-220048240a7050a98411ddbc6b1f82cf} | 0 ...n_date-20-e734eb4fc8894c9a2b2b9cdac4270fba | 1 + ...n_date-21-b60c0a3677591991152a0aa500bdce68 | 1 + ..._date-22-fe277bd0a30e016826d242fd5bd61714} | 0 ...n_date-23-7f014b494185e4c2a6048efb89131344 | 1 + ..._date-24-7ec1f3a845e2c49191460e15af30aa30} | 0 ...n_date-3-f8887dd18c21bf0306e293d463b3e1d7} | 0 ...n_date-4-f7c20c208e052305c1710a5e1e80c5c8} | 0 ...n_date-5-59355f4e222dcd4f77f51d15bd896f11} | 0 ...n_date-6-416ab10ac818d432cd3a81870d6e5164} | 0 ...on_date-7-1405c311915f27b0cc616c83d39eaacc | 3 + ...on_date-8-7703adfcfdd44c9250f9eba478004714 | 5 + ...on_date-9-a425c11c12c9ce4c9c43d4fbccee5347 | 1 + ...de_name-0-6ae3e0805cbfa967f50ad222bc33b772 | 0 ...de_name-1-4de8e998198c8df484e9102f60ba05c1 | 0 ...e_name-10-f1c90c8f4afd0c2ee66019e017997eb5 | 3 - ...e_name-11-94d637f6e5cee2771b9844438008a618 | 1 - ...de_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...de_name-6-f1c90c8f4afd0c2ee66019e017997eb5 | 3 - ...de_name-7-94d637f6e5cee2771b9844438008a618 | 1 - ...de_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 | 8 +- ...schema1-4-9b756f83973c37236e72f6927b1c02d7 | 10 +- ...schema1-5-52a518a4f7132598998c4f6781fd7634 | 8 +- ..._format-3-54d18742b4eab85edd1946ef139771b4 | 2 +- ...al_char-0-6ae3e0805cbfa967f50ad222bc33b772 | 0 ...al_char-1-4de8e998198c8df484e9102f60ba05c1 | 0 ...al_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...al_char-5-f1c90c8f4afd0c2ee66019e017997eb5 | 3 - ...al_char-6-94d637f6e5cee2771b9844438008a618 | 1 - ...al_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...al_char-8-f1c90c8f4afd0c2ee66019e017997eb5 | 3 - ...al_char-9-94d637f6e5cee2771b9844438008a618 | 1 - ...check-11-60aad884be613d18d3b89fca3b90dc94} | 0 ..._check-11-a1164f1770d2f787b520fbc3d345911a | 0 ..._check-2-3a6bb204d9524299f28adf1cc35d6f4d} | 0 ...e_check-2-5e857e1536264658caf0df9b7e28652f | 0 ...e_check-6-1f5f161f5e992a6aa7358dcbbffc5939 | 0 ..._check-6-663051c7106b8ee6913ca1b007157941} | 0 ...archar1-2-bca5c9edccc3a84e0a9ef92ebcbe746} | 0 ...archar1-2-fc9381cdfb786fc3b7b9b35df6003474 | 0 ...archar1-3-968759281b7d1bf0a60991ed04953b93 | 0 ...varchar1-3-9cec4b1d156f5a9cb587470b98de15} | 0 ...archar1-4-deb9b7715610152bda285a3a33b772ef | 0 ...rchar1-4-e9c4530e270db6d44cc54292e4eff680} | 0 ...rchar1-5-661aefd18c44c1eb02c2aaf8fab59f73} | 0 ...archar1-5-f77df9235261b945ca0a55e1afe2e5ce | 0 ...etadata-0-c25482149887659ec66d7534cad22f63 | 0 ...etadata-1-29f193b3e8def419733366c578e8a236 | 0 ...etadata-2-14033db9e60aa0bc4f64376c96195adf | 0 ...etadata-3-9bdf636332ca722093413523366efa86 | 0 ...etadata-4-e784348b7221bb26830cc1ebda69bdde | 1000 ------- ...eformat-0-c854b607353e810be297d3159be30da4 | 0 ...eformat-1-9fad934993b77eef15a5d10eb203a378 | 0 ...format-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 15 - ...format-11-606ad10de7caf7e65e09778f2673e712 | 25 - ...format-12-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 - ...format-13-c262e8c736b67119b9806f69eb492ef3 | 50 - ...format-14-da1b1887eb530c7e9d37667b99c9793f | 0 ...format-15-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format-16-7d619408a560b5c8d4a06dcd0ee106e5 | 15 - ...format-17-a488cb92e1388a7542d8787a22f4fb55 | 15 - ...format-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 15 - ...format-19-e3b55daf170a032dcc8ed12ee26ccf63 | 15 - ...eformat-2-7d619408a560b5c8d4a06dcd0ee106e5 | 15 - ...format-20-606ad10de7caf7e65e09778f2673e712 | 25 - ...format-21-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 - ...format-22-c56c391659f5701a9d2e8782a60f7f8a | 25 - ...format-23-c262e8c736b67119b9806f69eb492ef3 | 75 - ...format-24-fe564b4f68241ec5c5884e44a1306f4f | 75 - ...eformat-3-a488cb92e1388a7542d8787a22f4fb55 | 15 - ...eformat-4-606ad10de7caf7e65e09778f2673e712 | 25 - ...eformat-5-c262e8c736b67119b9806f69eb492ef3 | 25 - ...eformat-6-6c4f7b115f18953dcc7710fa97287459 | 0 ...eformat-7-f5f427b174dca478c14eddc371c0025a | 0 ...eformat-8-7d619408a560b5c8d4a06dcd0ee106e5 | 15 - ...eformat-9-a488cb92e1388a7542d8787a22f4fb55 | 15 - ...ormat10-0-66ee62178e3576fb38cb09800cb610bf | 1 - ...ormat10-1-198cb7d650a506ec3420b94b82a01375 | 0 ...ormat10-2-f723dedd396bd468107755b4495c1031 | 0 ...ormat10-3-c278fd699aa25809bdef310fb92f510e | 2 - ...ormat10-4-620cb6a290ef5297ac1d3a9ea776e2bf | 2 - ...ormat10-5-dd4c3f1636af9a7371edf7142abee088 | 0 ...ormat10-6-620cb6a290ef5297ac1d3a9ea776e2bf | 2 - ...ormat10-7-c278fd699aa25809bdef310fb92f510e | 2 - ...ormat11-1-b30db33650de0545cbcd53769ed7cf40 | 0 ...rmat11-10-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...rmat11-11-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat11-2-92bfcf88ca528eb6c9259142bf6541e5 | 0 ...ormat11-3-61f1abcdc66a64c11df85dded920d167 | 0 ...ormat11-4-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat11-5-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat11-6-ee7394c912991b8cd4401fb94942351f | 0 ...ormat11-7-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat11-8-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat11-9-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat12-1-b30db33650de0545cbcd53769ed7cf40 | 0 ...rmat12-10-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...rmat12-11-75a0aa2df39cbdc6a470b4c88803b42b | 4 - ...rmat12-12-3fc394a7a3d43a70968282ef1ee21dbd | 4 - ...rmat12-13-26a9a9cafa15d0223b934eba4777aea7 | 0 ...rmat12-14-d35f445501407d6fae5c3ad161fb2236 | 6 - ...rmat12-15-3fc394a7a3d43a70968282ef1ee21dbd | 6 - ...ormat12-2-92bfcf88ca528eb6c9259142bf6541e5 | 0 ...ormat12-3-61f1abcdc66a64c11df85dded920d167 | 0 ...ormat12-4-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat12-5-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat12-6-ee7394c912991b8cd4401fb94942351f | 0 ...ormat12-7-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat12-8-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat12-9-b8aed3dd7d7c151f5d96050c4058e71d | 0 ...format13-1-9a812f44c4c7a4c72b4be95139e6fd6 | 0 ...rmat13-10-5bc619cec1d519c1dc6423f841b1c0a4 | 1 - ...ormat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 | 0 ...ormat13-3-9b3e4a236550f1900a36566830b24024 | 0 ...ormat13-4-49cf189a09e11c2c635fbb574b89a2af | 0 ...ormat13-5-2ebe5fe98e830143b9571b13fe59dc0b | 0 ...ormat13-6-fa2f56078df18add8e5a77e538756488 | 0 ...ormat13-7-1d822cc037d9281ce172e2d5685b1495 | 0 ...ormat13-8-e4531456a7418952ec1d69e18bc8460b | 0 ...ormat13-9-8f6983fda271fba18d218c75b8fb2b3d | 1 - ...rmat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 | 0 ...rmat14-11-bbfb0a95274044dea4732e35c1d7ecbe | 0 ...rmat14-12-be84e8199b0a3b9f72e552018854ac15 | 1 - ...rmat14-15-dd6e9965d271bd35604059540c23d967 | 1 - ...rmat14-17-f4a3e39d5df18861e9fe67ef37af57e1 | 1 - ...rmat14-18-58080916a9f5883121bcaad719be0309 | 1 - ...ormat14-3-2683f9835169a568c1f03dae859d27d2 | 0 ...ormat14-4-2d2e1149d2f035017bb37bbfaad4def0 | 0 ...ormat14-5-61a8225e20f36e286b4d02501d2c80d9 | 0 ...ormat14-6-b5165befb75ebeed42f3e69d4d64375c | 0 ...ormat14-7-e438f7ec386b2ab19660e9da7260dd95 | 0 ...ormat14-8-f0e3ddaa1e6ea067444b1f540bfac293 | 0 ...ormat14-9-adeaa033260f16c5bc106e761e5fef8b | 0 ...ormat15-1-e1cf6c355de3ae8db7564b1676199117 | 0 ...rmat15-10-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...rmat15-11-662bece7f71ef854ca6e25f0eef5b830 | 0 ...rmat15-12-8250422b5ed16120ee33bd4fa69d3f47 | 3 - ...rmat15-13-3fc394a7a3d43a70968282ef1ee21dbd | 3 - ...ormat15-2-107d7c681b43611df056238be242127b | 0 ...ormat15-3-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat15-4-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat15-5-ee7394c912991b8cd4401fb94942351f | 0 ...ormat15-6-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat15-7-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat15-8-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat15-9-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat16-0-66ee62178e3576fb38cb09800cb610bf | 1 - ...ormat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 | 0 ...rmat16-10-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...rmat16-11-662bece7f71ef854ca6e25f0eef5b830 | 0 ...rmat16-12-8250422b5ed16120ee33bd4fa69d3f47 | 3 - ...rmat16-13-3fc394a7a3d43a70968282ef1ee21dbd | 3 - ...ormat16-2-107d7c681b43611df056238be242127b | 0 ...ormat16-3-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat16-4-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat16-5-ee7394c912991b8cd4401fb94942351f | 0 ...ormat16-6-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat16-7-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat16-8-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat16-9-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat17-0-7c49277a7217a147685d30e27822d273 | 0 ...ormat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 | 1 - ...format2-0-c854b607353e810be297d3159be30da4 | 0 ...format2-1-9fad934993b77eef15a5d10eb203a378 | 0 ...ormat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d | 0 ...ormat2-11-20a02894f5e9340e89b55a30bef252b7 | 75 - ...format2-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...format2-3-f5f427b174dca478c14eddc371c0025a | 0 ...format2-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...format2-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format2-6-32b36a6c3344c5fcca0ad6c93ffcab62 | 1 - ...format2-7-561ef0dbccfcbfbb0c75f33ebfd4203d | 0 ...format2-8-20a02894f5e9340e89b55a30bef252b7 | 75 - ...format2-9-50131c0ba7b7a6b65c789a5a8497bada | 1 - ...format3-0-c854b607353e810be297d3159be30da4 | 0 ...format3-1-6c4f7b115f18953dcc7710fa97287459 | 0 ...ormat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 - ...format3-2-f5f427b174dca478c14eddc371c0025a | 0 ...format3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 14 - ...format3-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...format3-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format3-6-e3b55daf170a032dcc8ed12ee26ccf63 | 14 - ...format3-7-c56c391659f5701a9d2e8782a60f7f8a | 25 - ...format3-8-f5f427b174dca478c14eddc371c0025a | 0 ...format3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 14 - ...format5-0-66ee62178e3576fb38cb09800cb610bf | 2 +- ...format5-6-a0e23b26ee1777ccc8947fb5eb1e8745 | 2 +- ...format5-8-a0e23b26ee1777ccc8947fb5eb1e8745 | 2 +- ...format6-0-66ee62178e3576fb38cb09800cb610bf | 2 +- ...format6-6-e95296c9f7056b0075007c61d4e5e92f | 2 +- ...format6-7-4758d41d052eba37a9acd90c2dbc58f0 | 2 +- ...format7-0-66ee62178e3576fb38cb09800cb610bf | 2 +- ...format7-4-a34505bd397bb2a66e46408d1dfb6bf2 | 2 +- ...format7-5-f2c42f1f32eb3cb300420fb36cbf2362 | 2 +- ...format8-0-66ee62178e3576fb38cb09800cb610bf | 1 - ...format8-1-22e3d59a0423473051535684bca72b27 | 0 ...format8-2-55ae9fbf6daa36225dd386e34025dd38 | 0 ...format8-3-c561806d8f9ad419dc9b17ae995aab68 | 0 ...format8-4-ae71ce67b5d4a91bce1b34acde830268 | 0 ...format8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e | 0 ...format8-6-ae6a5016b6a6ace6b6f2576a4c666b4e | 0 ...format8-7-d1a5913edaaec9654edb333e8207f57b | 20 - ...format8-8-624b059dfaa86e2c78f065169de325cd | 20 - ...format9-0-66ee62178e3576fb38cb09800cb610bf | 2 +- ...format9-5-d1a5913edaaec9654edb333e8207f57b | 2 +- ...format9-6-624b059dfaa86e2c78f065169de325cd | 2 +- .../ppd1-0-ae225e86c2ae20519ffdf23190454161 | 2 +- .../ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- .../ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 | 2 +- .../ppd1-4-145c2779dadb5bd921dc2baac608b803 | 2 +- .../ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 | 2 +- .../ppd2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- .../ppd2-1-145c2779dadb5bd921dc2baac608b803 | 2 +- .../ppd2-3-fccdc7a24b626308d9ec17608f36564b | 2 +- .../ppd2-6-ae225e86c2ae20519ffdf23190454161 | 2 +- .../ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- .../ppd2-9-fccdc7a24b626308d9ec17608f36564b | 2 +- ...nt_expr-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...nt_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...nt_expr-5-7da8c74586c99d96254f6f031bcaeb1c | 2 +- ...nt_expr-6-145c2779dadb5bd921dc2baac608b803 | 2 +- ...nt_expr-9-7da8c74586c99d96254f6f031bcaeb1c | 2 +- ...t_where-1-84c951641740895ca1c8ddc098805da5 | 2 +- ...ppd_gby-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d | 2 +- ...ppd_gby-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d | 2 +- ...pd_gby2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...pd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...ppd_gby2-3-321628d4c52f6992f2680a3a162f19f | 2 +- ...pd_gby2-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...ppd_gby2-6-321628d4c52f6992f2680a3a162f19f | 2 +- ...by_join-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...by_join-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...by_join-3-145c2779dadb5bd921dc2baac608b803 | 2 +- ...pd_join-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...pd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...pd_join-3-aab5d092ca17ed23ee71d3a6ef653998 | 2 +- ...pd_join-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...pd_join-6-aab5d092ca17ed23ee71d3a6ef653998 | 2 +- ...d_join2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...d_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...d_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 | 2 +- ...d_join2-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...d_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 | 2 +- ...d_join3-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...d_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...d_join3-3-42cd793c031af1f7961d7b5e237de76b | 2 +- ...d_join3-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...d_join3-6-42cd793c031af1f7961d7b5e237de76b | 2 +- ...r_join1-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...r_join1-3-9dfd333c14f82fd71c213e1d39c83524 | 2 +- ...r_join1-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...r_join1-6-9dfd333c14f82fd71c213e1d39c83524 | 2 +- ...r_join2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...r_join2-3-5340cd2d88dcf41dc18119389a475e36 | 2 +- ...r_join2-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...r_join2-6-5340cd2d88dcf41dc18119389a475e36 | 2 +- ...r_join3-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...r_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 | 2 +- ...r_join3-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...r_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 | 2 +- ...r_join4-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...r_join4-3-ac57dc2a7151f20029d6a97049d6eebe | 2 +- ...r_join4-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...r_join4-6-ac57dc2a7151f20029d6a97049d6eebe | 2 +- ...r_join5-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join5-1-145c2779dadb5bd921dc2baac608b803 | 2 +- ..._random-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ..._random-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ..._random-3-145c2779dadb5bd921dc2baac608b803 | 2 +- ...ansform-0-ae225e86c2ae20519ffdf23190454161 | 1 - ...ansform-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 - ...ansform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 | 0 ...ansform-3-5d0e4236af589d3e63a8dd84e663b745 | 84 - ...ansform-4-145c2779dadb5bd921dc2baac608b803 | 1 - ...ansform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 | 0 ...ansform-6-5d0e4236af589d3e63a8dd84e663b745 | 84 - ...udf_col-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...udf_col-6-145c2779dadb5bd921dc2baac608b803 | 2 +- ...d_union-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...d_union-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...d_union-3-678c8197f458b459171c266f7431683e | 2 +- ...d_union-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...d_union-6-678c8197f458b459171c266f7431683e | 2 +- ...on_view-0-d680c8ac7e5121c6971458a9022c00b9 | 0 ...on_view-1-bfb5ca0dd2d5f070ce359790f8b91480 | 0 ...n_view-10-a74a5ff32204d842175b3d880477178f | 0 ...n_view-11-745e750f4f4a36af27e87338a979240c | 0 ...on_view-2-ac54e78582e6bd5d5533a0f3bfb51369 | 0 ...on_view-3-f6fd9a5271e172e4a65ea57aaa41d512 | 0 ...on_view-4-19cd3ea4e274befd809e4aad21da8d50 | 0 ...on_view-5-443c0979b586f6a6dfb0dc5d28cd5124 | 0 ...on_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 | 0 ...on_view-7-aab9d79f65d6edfc1cae88a14b8e106c | 0 ...on_view-8-e3ee46daeac605b05c7ada97b3c43578 | 0 ...on_view-9-8a77c30b6b07717d1c0bee971fb0132c | 0 .../ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 | 0 .../ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 | 36 - .../ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 | 0 .../ppd_vc-3-be78760949abf728580442e9e37a3ce7 | 36 - ...shdown-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...ushdown-0-855b235f9c760ba9d6d0200bfd1ad08b | 0 ...shdown-1-855b235f9c760ba9d6d0200bfd1ad08b} | 0 ...ushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 | 0 ...shdown-10-178be64f122542983ff4384df4bb1530 | 0 ...hdown-10-2957fd9b211cee5f0372525a1de55c19} | 0 ...shdown-11-3828393aa33a55cf1aea707e1be0a452 | 0 ...hdown-11-b8d6f0ffc8294497c792b26958adee45} | 0 ...shdown-12-5affd35b94b0de3fb2a43f8729526055 | 0 ...hdown-12-ab1b9a0cdd9586c96d8856a9d632b89c} | 0 ...hdown-13-2c316c67fd494d878fbbea107d283c3b} | 0 ...shdown-13-2c64f5abe8c23957d7f0602e9c257cd1 | 0 ...hdown-14-53b4be82a1538844d03b200429efa02b} | 0 ...shdown-14-c3188230751166d9d90082cd357ecb0c | 0 ...hdown-15-71ab93d38ed2908069091c7c8cc0aba1} | 0 ...shdown-15-e52a39743f82af47902341a7ecd38afb | 0 ...shdown-16-70bef3ba441873327e188ff2ec6d68ae | 0 ...hdown-16-855cb54d28034fdb20a3615ee0918d63} | 0 ...hdown-17-d3ccf2722a8b7281fcee61b2544772c8} | 0 ...hdown-18-fcd10fbfc55a2c0aa843fe618f9613c6} | 0 ...hdown-19-ff36e3978571ac05f11e8322c024e4b6} | 0 ...ushdown-2-5eede4874e678021938683fc2f5dc900 | 0 ...shdown-2-f689aaf15b08c433d1e93ce977a9b6d5} | 0 ...hdown-20-481005cf034ef3d7b998da32eb82aa9a} | 0 ...hdown-21-9073825e8b9804331f780980d1f9fa92} | 0 ...hdown-22-9cb28f0d3a434c9d1ab039192906ec9d} | 0 ...hdown-23-678f50025924fee7b59f66e2abdb472d} | 0 ...hdown-24-4d5bfa800ba434f464b07bf9b5d39f20} | 0 ...hdown-25-22663f09ea1c1bc303756067e84df5a7} | 0 ...shdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b | 0 ...hdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b} | 0 ...hdown-27-8065c18e387fd8bffae19a80af8dc1d4} | 0 ...hdown-28-b72de558c88ae91460989938000e0d27} | 0 ...hdown-29-7217735d357770af4ffee730e4e9add4} | 0 ...shdown-3-5eede4874e678021938683fc2f5dc900} | 0 ...ushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 | 0 ...hdown-30-630e2f7918b7727fc4ca057fa21e2eea} | 0 ...hdown-31-2e062414293b643ea4d7f6df92f939e4} | 0 ...hdown-32-5eba4bf10315099129eae319d73636cf} | 0 ...hdown-33-35af0585a4f98bc222c786688cb6de6b} | 0 ...hdown-34-a5594625510703427ab8bae5d0563f73} | 0 ...shdown-4-c5c542f8ee81cd0afd44e67fc7b4d306} | 0 ...ushdown-4-f54bebec398f0fdfdbc0393123dba234 | 0 ...pushdown-5-78af12432bcbf99d4a0d41c25f964de | 0 ...shdown-5-f54bebec398f0fdfdbc0393123dba234} | 0 ...ushdown-6-58724fbe96a0b3deceef20c8cc5e318d | 0 ...ushdown-6-78af12432bcbf99d4a0d41c25f964de} | 0 ...ushdown-7-14570f946e75924d7926c809485951d1 | 0 ...shdown-7-58724fbe96a0b3deceef20c8cc5e318d} | 0 ...shdown-8-14570f946e75924d7926c809485951d1} | 0 ...ushdown-8-3ba325662296fc455f07f1c835495e4c | 0 ...shdown-9-3ba325662296fc455f07f1c835495e4c} | 0 ...pushdown-9-4375f17bed264d5554a67d119fa5dd1 | 0 ...hdown2-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...shdown2-0-855b235f9c760ba9d6d0200bfd1ad08b | 0 ...shdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 | 0 ...hdown2-1-855b235f9c760ba9d6d0200bfd1ad08b} | 0 ...hdown2-10-4fa4ba4c739b6f44975d41e4705d4389 | 1 + ...hdown2-10-ab3e46183487096745d7d90e3020e94c | 0 ...hdown2-11-680316eba447eb4649530fdc1c37d95b | 0 ...down2-11-ab3e46183487096745d7d90e3020e94c} | 0 ...hdown2-12-720582e599a974ee9ca46b653678a14a | 0 ...down2-12-b10b9e14f5a7a7a92c0c68df6dbc656a} | 0 ...down2-13-154f9859bd0822e287fbfdff12fd45ff} | 0 ...hdown2-13-bd9067aeee8884a32db921b6d478f806 | 0 ...hdown2-14-8d01597374157f2d3d066840983ba1f8 | 1 - ...down2-14-cf4a30b5c8329d8d79ddf762f318fbb3} | 0 ...shdown2-15-5614065e1b8e709f68be4fa67666f41 | 1 - ...hdown2-15-8d01597374157f2d3d066840983ba1f8 | 1 + ...hdown2-16-3a67618e47c977f58c9dd8f4b9a576eb | 1 - ...shdown2-16-5614065e1b8e709f68be4fa67666f41 | 1 + ...hdown2-17-3a67618e47c977f58c9dd8f4b9a576eb | 1 + ...hdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe | 2 - ...hdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe | 2 + ...hdown2-18-e2c7e9b01ec95dfcc685827e24d66775 | 3 - ...hdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 | 1 - ...hdown2-19-e2c7e9b01ec95dfcc685827e24d66775 | 3 + ...shdown2-2-efd3e4c333d4efb81218df8921e58f9e | 0 ...hdown2-2-ffa167b63d612a4986d02f5c0623ea7b} | 0 ...hdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 | 1 + ...hdown2-20-ece6fe0efc1e658b36ddc10f0653d229 | 1 - ...hdown2-21-ece6fe0efc1e658b36ddc10f0653d229 | 1 + ...shdown2-3-1886675984606b6c972c4a26dca6fd2c | 1 - ...hdown2-3-c7753746c190414723d66a8f876499c7} | 0 ...shdown2-4-1886675984606b6c972c4a26dca6fd2c | 1 + ...shdown2-4-33b137b28e7246ec3c2acb937c638910 | 1 - ...shdown2-5-33b137b28e7246ec3c2acb937c638910 | 1 + ...shdown2-5-b5a2518af801f95fe52a75dfc1d3e867 | 0 ...shdown2-6-96059d8042ccb2ce355541daf9893954 | 0 ...hdown2-6-b5a2518af801f95fe52a75dfc1d3e867} | 0 ...shdown2-7-93922fe8d88643e1b8aa7e3ef4195404 | 0 ...hdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b} | 0 ...hdown2-8-4507a3f200b3ce384191c91acd324dc7} | 0 ...shdown2-8-c86940e70f705f50e9091c257ee2bb40 | 1 - ...shdown2-9-4fa4ba4c739b6f44975d41e4705d4389 | 1 - ...shdown2-9-c86940e70f705f50e9091c257ee2bb40 | 1 + ...shdown3-0-df2401785dfa257de49c3ad80b0f480a | 2 +- ...shdown3-2-e879051803d0b64139e703e40fb007d0 | 2 +- ...shdown3-4-a1c18483e5f5d2fe351be09af658acbb | 2 +- ...shdown3-6-67118904e6cc8e9f5210fab88e87fb8f | 2 +- ..._header-0-860e298a0b70e7a531431e9386ddc0e7 | 1 - ..._header-1-8540676fc16ac91f3629c40f393a890a | 10 - ..._header-2-5cff10d4b561206e7e0b2e81d862ff93 | 10 - ..._header-3-e86d559aeb84a4cc017a103182c22bfb | 0 ...gress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 | 2 +- ...gress_1-2-70ba81c09588aa586e62ebaee2af685f | 0 ...ress_1-2-b6c8c8fc9df98af4dead5efabf5f162c} | 0 ...gress_1-3-43d286eebddaee26cf95f26e62a75fe4 | 2 +- ...ctmode-19-b6e156f104768706aa587b762a9d4d18 | 8 +- ...ctmode-21-a31442a7d6c6950c137856ab861d622d | 8 +- ...ctmode-23-a31442a7d6c6950c137856ab861d622d | 8 +- ...ctmode-26-a31442a7d6c6950c137856ab861d622d | 8 +- ...ctmode-28-a31442a7d6c6950c137856ab861d622d | 8 +- ...ctmode-32-b6e156f104768706aa587b762a9d4d18 | 8 +- ...ectmode-6-e2d1fc9c94e0d3597f393aec99850510 | 4 +- ...ectmode-8-e2d1fc9c94e0d3597f393aec99850510 | 4 +- ...push_or-5-c94def4b18b9c8c00e7a93eb19ec694a | 2 +- ...th_semi-0-3731ce715b60549c17b7993927d48436 | 2 +- ...th_semi-1-3f53ec3b276b32cf81729433e47010cb | 2 +- ...th_semi-2-bf8cb175f9b13fcc23ba46be674b5767 | 2 +- ...quote2-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 .../quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 | 0 .../quote2-1-34f3c423b2fb1f0b11457f45a60042b9 | 1 - ...quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118} | 0 .../quote2-2-34f3c423b2fb1f0b11457f45a60042b9 | 1 + ...as.attr-0-97b3c408090f758257e4bd20597a525e | 2 +- ...pruner1-0-a7e4414330751eb8ad486bb11643f64d | 0 ...pruner1-1-11cdebc422d7a0e7b257279ac9524321 | 56 - ...pruner2-0-b9598847d77e2c425423f51d755380e8 | 0 ...pruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 | 0 ...pruner2-2-db276de57ad86e8880037336886cd557 | 0 ...pruner2-3-e4419c33287ca1f48a43f61cca5b5928 | 116 - ...bigdata-0-e011be1172043c0c6d0fd2c0e89f361e | 1 - ...bigdata-1-6d0d6f4de136f56ab91987e19df8e178 | 1 - ...bigdata-2-3688b45adbdb190d58799c0b6d601055 | 1 - ...bigdata-3-fc0c054cdfbf5c130532e139d497866a | 0 ...olumnar-2-b2f56374f2ea8a967b38e77b57138d4b | 4 +- ..._merge1-0-d877ca1eefa4344eae30ad3ef2039b00 | 1 - ..._merge1-1-a4fb8359a2179ec70777aad6366071b7 | 1 - ...merge1-10-f94fdb0b79dcf73989e6fbce87355753 | 1 - ...merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 | 0 ...merge1-12-150cb190dc2343a747ea89298eb6352c | 0 ...merge1-13-af7b1e79eb40854ea82d61debaafcf40 | 1 - ..._merge1-14-4547f75ed3cb94914c2d025c1e057b5 | 1 - ...merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 | 0 ...merge1-16-c198c437f48c3844d8d0ace881b3107e | 0 ..._merge1-2-690b2898f94ef32f01ded0ddb737a056 | 1 - ..._merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee | 1 - ..._merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 | 0 ..._merge1-5-c198c437f48c3844d8d0ace881b3107e | 0 ..._merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 | 0 ..._merge1-7-8c76c8d41133f29366359c308b0c9cc0 | 0 ..._merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f | 0 ..._merge1-9-780b166bd3fcc932cb217273b0636d63 | 0 ..._merge2-0-f94fdb0b79dcf73989e6fbce87355753 | 1 - ..._merge2-1-a4fb8359a2179ec70777aad6366071b7 | 1 - ...merge2-10-5b4fb8038f522877101a4e429f082f11 | 0 ..._merge2-2-690b2898f94ef32f01ded0ddb737a056 | 1 - ..._merge2-3-778b659dba30ece782a956d87b1a31eb | 1 - ..._merge2-4-5b4fb8038f522877101a4e429f082f11 | 0 ..._merge2-5-78b97f09b49452e054cba2ae461813c3 | 0 ..._merge2-6-d5a91ca1ff9039b2dc55af8735d0249a | 0 ..._merge2-7-805fc662f9b3b7a587d1043b659e2424 | 0 ..._merge2-8-afd119f412a66bdf5cd1c74ae8965685 | 1 - ..._merge2-9-351c25ee0ca991ca9e0dbafa873b7592 | 1 - ..._merge3-0-f94fdb0b79dcf73989e6fbce87355753 | 1 - ..._merge3-1-690b2898f94ef32f01ded0ddb737a056 | 1 - ...merge3-10-1486feb77f13bb9a0ed606fe795ef686 | 0 ...merge3-11-7674266b00c52a2b6755910ea0793b69 | 1 - ...merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f | 1 - ...merge3-13-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...merge3-14-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge3-2-778b659dba30ece782a956d87b1a31eb | 1 - ..._merge3-3-d57efd24ca36f282a37be5d0bf6452e6 | 0 ..._merge3-4-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 | 0 ..._merge3-6-e4d1c54cf6a802eef3342bec2505f69b | 0 ..._merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 | 0 ...e_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 | 0 ..._merge3-9-7c38c42af102069e08a85a27a735eb61 | 0 ..._merge4-0-f94fdb0b79dcf73989e6fbce87355753 | 1 - ..._merge4-1-690b2898f94ef32f01ded0ddb737a056 | 1 - ...merge4-10-1486feb77f13bb9a0ed606fe795ef686 | 0 ...merge4-11-7674266b00c52a2b6755910ea0793b69 | 1 - ...merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f | 1 - ...merge4-13-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...merge4-14-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge4-2-778b659dba30ece782a956d87b1a31eb | 1 - ..._merge4-3-d57efd24ca36f282a37be5d0bf6452e6 | 0 ..._merge4-4-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge4-5-18f216bf1ea33debc2576ac85ac1a449 | 0 ..._merge4-6-3e292c7212dd02d7d6000837e09a42f9 | 0 ..._merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 | 0 ...e_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 | 0 ..._merge4-9-7c38c42af102069e08a85a27a735eb61 | 0 ...l_value-2-e721b8138774fdefca7171d1051841ee | 2 +- ...ll_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 | 2 +- ...uptions-1-a9f718201b71fa855fb81a29e1d7bd3b | 2 +- ...uptions-2-4a9a9175758ef576804c8b7309b019e8 | 2 +- ...uptions-3-ea607fbed28d20e5726f4501285d698d | 2 +- ...uptions-5-6a3af12e36cec853c876a2cbae61c23a | 2 +- ...uptions-6-718032defb864225dd664b1719f3b590 | 2 +- ...d table-0-ce3797dc14a603cba2a5e58c8612de5b | 1 - ...ive_dir-0-fb096f0f4ecc530357ad76ae0353d338 | 0 ...de_join-0-aa047b3a8b40b68b93c4ad11e173c767 | 2 +- ...de_join-1-24ca942f094b14b92086305cc125e833 | 2 +- ...gex_col-0-ac78bd83c9aa538c2827598dd6007a69 | 0 ...gex_col-1-42751bfc3f1e83e7a014db9272d597db | 0 ...egex_col-2-21564f64cdfd46098e1254380490701 | 0 ...gex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 | 0 ...gex_col-4-daf9d3ca22b243870a138ba90d1593c4 | 0 ..._script-0-4e2e94d649b1792c5bd3fd879349ef92 | 1 - ..._script-1-e168f471980470d93b790702a70238fa | 1 - ..._script-2-a19a19272149c732977c37e043910505 | 1 - ...te_script-3-4eb54a664e549614d56ca088c8867d | 0 ..._column-1-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-11-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-13-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-15-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-17-9c36cac1372650b703400c60dd29042c | 2 +- ...column-21-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-23-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-25-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-27-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-29-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ..._column-3-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-31-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-33-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-35-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ..._column-5-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ..._column-7-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ..._column-9-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...ocation-0-5c73d46fb91e9d4b3dc916622df09290 | 1 - ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...oin_ppr-2-49b8b038ed8f5513405071c532967c47 | 2 +- ...oin_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 | 2 +- ...oin_ppr-6-2edf8e4ca99543089fdacbf506caa94f | 2 +- ...oin_ppr-8-b07ad21f47a74162f438abf579675f8e | 2 +- ...sample2-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...sample2-1-a1d8184eab25b242a961533cc016efd1 | 0 ...de_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...de_hook-1-2b1df88619e34f221d39598b5cd73283 | 1 - ...e_hook-10-60eadbb52f8857830a3034952c631ace | 0 ...e_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 | 0 ...e_hook-12-60018cae9a0476dc6a0ab4264310edb5 | 0 ...de_hook-2-7562d4fee13f3ba935a2e824f86a4224 | 1 - ...de_hook-3-bdb30a5d6887ee4fb089f8676313eafd | 1 - ...de_hook-4-10713b30ecb3c88acdd775bf9628c38c | 1 - ...de_hook-5-bab89dfffa77258e34a595e0e79986e3 | 1 - ...de_hook-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...de_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f | 0 ...de_hook-8-f9dd797f1c90e2108cfee585f443c132 | 0 ...de_hook-9-22fdd8380f2652de2492b34a425d46d7 | 0 ...adoop20-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...adoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...doop20-10-22fdd8380f2652de2492b34a425d46d7 | 0 ...doop20-11-60eadbb52f8857830a3034952c631ace | 0 ...doop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 | 0 ...doop20-13-60018cae9a0476dc6a0ab4264310edb5 | 0 ...adoop20-2-2b1df88619e34f221d39598b5cd73283 | 1 - ...adoop20-3-7562d4fee13f3ba935a2e824f86a4224 | 1 - ...adoop20-4-bdb30a5d6887ee4fb089f8676313eafd | 1 - ...adoop20-5-10713b30ecb3c88acdd775bf9628c38c | 1 - ...adoop20-6-bab89dfffa77258e34a595e0e79986e3 | 1 - ...adoop20-7-6f53d5613262d393d82d159ec5dc16dc | 1 - ...adoop20-8-7a45282169e5a15d70ae0afb9e67ec9a | 0 ...adoop20-9-f9dd797f1c90e2108cfee585f443c132 | 0 ...nv_var1-0-16015162957e1d8e0ef586b44b276f64 | 2 - ...nv_var2-0-e5c4893b2ff412f9df0632328d404cef | 1 - ...nv_var2-1-81cb3e18ab89c533a253defff35e90f0 | 2 - ...pt_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 | 1 - ...pt_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 | 0 ...pt_pipe-2-3bf368261963ee10883e97c7746796b5 | 0 ...pt_pipe-3-afe5db164ccf986c5badd0655e009ea1 | 0 ...pt_pipe-4-7fe60c2fcca928a497252d99436b513f | 1 - ...ptfile1-0-43d53504df013e6b35f81811138a167a | 1 - ...ptfile1-1-89cf592f66b7276add70eb2c7689aa34 | 0 ...ptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 | 1 - ...d table-0-304c4992f5517febd10f43c57df4da49 | 11 + ...omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 | 2 +- ...ote_and-3-683007138a712792041ef3c8b84e914e | 2 +- ...ote_and-5-1a5a68a098bfb9c93b76a458a9faf232 | 2 +- ...ote_not-3-683007138a712792041ef3c8b84e914e | 2 +- ...ote_not-5-ceada0d3fa65bb2ec65154e797332cde | 2 +- ...uote_or-3-683007138a712792041ef3c8b84e914e | 2 +- ...uote_or-5-55cd874f705673f9de6ec8e3643c760f | 2 +- ...e_regex-0-60462d14b99bb445b02800e9fb22760e | 22 - ..._regex-10-c5b3ec90419a40660e5f83736241c429 | 4 +- ...e_regex-2-9d00484beaee46cf72b154a1351aeee9 | 0 ..._regex-2-e84d30fcc6cf11e82b54ea63e7d1d611} | 0 ..._regex-3-3ee9e78ff563d6b48741a41885f92c81} | 0 ...e_regex-3-817190d8871b70611483cd2abe2e55dc | 0 ...e_regex-7-4db287576a17c0963219ca63ee0b20e0 | 22 - ..._regex-7-bf456bcf6be7334488424dfeadf27d75} | 0 ..._regex-8-a4cf34af32b83e40e5c8b6d083938b54} | 0 ...e_regex-8-c429ee76b751e674992f61a29c95af77 | 0 ...e_regex-9-f0e8d394ad18dcbd381792fe9bd8894b | 0 ..._regex-9-f1175f3322abec6f258dd49a5905bce0} | 0 ..._schema-1-e8c6de8cd50be582d78c9a8244cd92a1 | 6 +- ..._schema-3-738e1d72a19c3db37ded87ca2fb148fa | 6 +- ...ble_sub-0-b0b2ec9de0599f8dc2eb160299a2699d | 2 +- ...le_sub-10-bf5d1e710ce0974a67b4084aaa868f67 | 2 +- ...ble_sub-3-266170978f88a14c20c3944bfb55f5c7 | 2 +- ...ble_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 | 2 +- ...ble_sub-5-ee3a31bb9bb742f90daea98b290c34cd | 2 +- ...ble_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b | 2 +- ...ble_sub-8-f321516e93eb0206453465a9b85cf67d | 2 +- ...ble_sub-9-fbb54d457caeaafce723856429bbc0b2 | 2 +- ...e_alter-1-2a91d52719cf4552ebeb867204552a26 | 10 +- ...e_alter-3-2a91d52719cf4552ebeb867204552a26 | 19 +- ...e_alter-5-2a91d52719cf4552ebeb867204552a26 | 19 +- ...e_alter-7-2a91d52719cf4552ebeb867204552a26 | 19 +- ...e_alter-9-2a91d52719cf4552ebeb867204552a26 | 19 +- ...b_table-4-b585371b624cbab2616a49f553a870a0 | 10 +- ...limited-0-97228478b9925f06726ceebb6571bf34 | 0 ...limited-1-2a91d52719cf4552ebeb867204552a26 | 12 +- ...itioned-1-2a91d52719cf4552ebeb867204552a26 | 12 +- ...e_serde-1-2a91d52719cf4552ebeb867204552a26 | 17 +- ...e_serde-4-2a91d52719cf4552ebeb867204552a26 | 10 +- ...le_view-1-1e931ea3fa6065107859ffbb29bb0ed7 | 2 +- ..._quotes-0-65fee14fcf58502241f0772b21096780 | 2 +- ..._quotes-1-26b98b2901556449d5431d731aaa642d | 2 +- ..._quotes-2-f795383fcecedf7266cd7aed8283cec3 | 2 +- ...c_quotes-3-7bc53505a4e6587132870d8d0a704d2 | 2 +- ...nctions-0-45a7762c39f1b0f26f076220e2764043 | 5 +- ...nctions-1-4a6f611305f58bdbafb2fd89ec62d797 | 4 +- ...nctions-2-97cbada21ad9efda7ce9de5891deca7c | 4 +- ...nctions-3-86945c60aed23626c43b507ee4ee6049 | 2 +- ...nctions-4-4deaa213aff83575bbaf859f79bfdd48 | 2 +- ...e_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 | 0 ...e_cases-1-3f42728fb0083beb962f07c43dd9c9b7 | 0 ..._cases-10-d759a63f08c878456c3401626f253ff5 | 0 ..._cases-11-afe66851d1cdc5d8a8a0d21c5705a59e | 0 ..._cases-12-f2dd38f0a56cd2137c9e1b870271550b | 16 - ..._cases-13-a9224a7a0012e407da67041bf680d490 | 4 - ..._cases-14-9d121385c5ab32d659dc7d0374ae8d6e | 16 - ..._cases-15-b032f4869c43d1278a890328d0591d5d | 0 ..._cases-16-8b2dc53c795584e1e23a8d631c82b43f | 0 ..._cases-17-c93fd07893f47b712165725c78d95555 | 0 ..._cases-18-9acff7f5096cdafc92aa2ddb0f296f83 | 0 ..._cases-19-3f42728fb0083beb962f07c43dd9c9b7 | 0 ...e_cases-2-e90c14f9edaeab8cf4540e9a35b11546 | 0 ...e_cases-3-aa5935155586821fb35e17156c8d8460 | 0 ...e_cases-4-6eb587e2751942de625c9229872ca0dc | 0 ...e_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 | 0 ...e_cases-6-ae97a64481efe733a19007ed400925bc | 0 ...e_cases-7-34016fb262ce768de45ec1b7693fd6c8 | 0 ...e_cases-8-16d39297488db165145e1546c4cb222c | 0 ...e_cases-9-fba02256147a1a753d40f56825449471 | 0 ..._syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec | 0 ..._syntax-1-fe3da04846e702e0bbe22e05b136a3b3 | 0 ...syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec | 0 ..._syntax-2-8207d7ca034ed62b9fb2c727497c22b3 | 0 ..._syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 | 0 ..._syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 | 16 - ..._syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 | 1 - ..._syntax-6-de64752733e0dcae32e692d2ad38e3d4 | 16 - ..._syntax-7-991839c8e50c5b4e490ec7faf2228d58 | 1 - ..._syntax-8-8c10f994f51bce851ecb0acee65ced7e | 16 - ..._syntax-9-f8385127c6405a2c9e48b5988184b515 | 4 - ...titions-0-73d0fdcea0bd9b828cbc3c2e88acb51a | 2 +- ...itions-1-e69b801a3c6c5f6692050bcdb0e31db9} | 0 ...titions-1-e94d4100cb64c67f1127b4e255d28ae0 | 2 - ...rtitions-2-a1bde7c2c040b4d45ddceac9983c2ca | 2 - ...titions-2-e94d4100cb64c67f1127b4e255d28ae0 | 2 + ...titions-3-9e3f80cb6ed9883c715ea8aa3f391d70 | 1 - ...titions-3-a1bde7c2c040b4d45ddceac9983c2ca} | 0 ...titions-4-9e3f80cb6ed9883c715ea8aa3f391d70 | 1 + ..._tables-0-679cc07f8475a20b49927a5bbbd3d702 | 0 ..._tables-1-ac1c8cca812353544d3f7dead5d033ce | 0 ...tables-10-643b24446d74450c0f83144b1d0ec433 | 0 ...tables-11-3f9a7f993510123059493826470f78f7 | 0 ...tables-12-c31d2f4105ec3714cfc55eef68d3f60c | 3 - ...tables-13-f72d7ab6f3f04161ce2b8f8332244657 | 4 - ...tables-14-26ca9b5509544ebac182d8aa4264ff1c | 3 - ...tables-15-72a95694f749cb3d5760a69083e9cafe | 4 - ...tables-16-dfd802554b6764b3105b6fd8dbb8e173 | 0 ...tables-17-49777c49d2627373ed5e459c4848c9ab | 0 ...tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 | 0 ...tables-19-695a68c82308540eba1d0a04e032cf39 | 0 ..._tables-2-c96604d76bcb3721d5e5a327cac0d5e5 | 18 - ...tables-20-691b4e6664e6d435233ea4e8c3b585d5 | 0 ...tables-21-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...tables-22-274454ebeb7f98690a3c152617a0e391 | 1 - ..._tables-3-a56f51be557c6f636f99fafdbbbbcd9c | 2 - ..._tables-4-743d585ec3da5fbb927581cd0683ae35 | 18 - ..._tables-5-c685b358b604bf3ef980a78d9178d87d | 2 - ..._tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 | 0 ..._tables-7-a62fc229d241303bffb29b34ad125f8c | 0 ..._tables-8-691b4e6664e6d435233ea4e8c3b585d5 | 0 ..._tables-9-64c9bf0618541518f2ba30ec24a94423 | 0 ...estatus-0-4fa957197c8678b0a3a64d8f4f6da1fa | 14 - ...estatus-1-4c31924711bdb64603a14ce57da86ab7 | 14 - ...estatus-2-ecddce523f2af516700677a051581330 | 14 - ...estatus-3-dccfbc8b5a223a9fe47120ca771ee61d | 14 - ...estatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 | 0 ...lestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 | 0 ...estatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 | 0 ...estatus-7-f7b9148c16045269344c5d74fb8a449c | 0 ...estatus-8-1cd5acb4091d916e5e18948a39979b51 | 14 - ...estatus-9-e3cc1823000abb51d2635e0c824e81a9 | 14 - ...perties-4-6c63215ea599f6533666c4d70606b139 | 12 +- ...owparts-0-593619bb962b318b82896658deaea1f1 | 17 - ...owparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a | 4 - ...emove_1-0-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...emove_1-1-114600d46ae86edcb66a500b4cac657d | 1 - ...move_1-10-fa00cf008a039908eec64ad5dd415c5b | 0 ...move_1-11-4abc4f450a58ccdd0df2e345f1276979 | 0 ...move_1-12-dd683e148baed6b27642eebacaa87a4f | 0 ...move_1-13-43d53504df013e6b35f81811138a167a | 1 - ...emove_1-2-cafed8ca348b243372b9114910be1557 | 1 - ...remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_1-4-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...emove_1-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_1-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_1-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_1-8-daf10744f465e055b35809a528135370 | 0 ...emove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...emove_2-0-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...emove_2-1-114600d46ae86edcb66a500b4cac657d | 1 - ...move_2-10-bebf0a312f3110d0b518153543030f06 | 0 ...move_2-11-4abc4f450a58ccdd0df2e345f1276979 | 0 ...move_2-12-fa53198d9eecb9d274b09d4351b9274e | 0 ...move_2-13-3fda17e4414d191f837631438a19e700 | 0 ...move_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 | 0 ...move_2-15-43d53504df013e6b35f81811138a167a | 1 - ...emove_2-2-cafed8ca348b243372b9114910be1557 | 1 - ...remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_2-4-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...emove_2-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_2-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_2-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_2-8-c64266431d312784ebc7b9ca07ab5188 | 0 ...emove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt1-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt1-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...oinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt1-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...oinopt1-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt1-7-f66c3935651d3cc5fef7d0284e123614 | 0 ...inopt10-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt10-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt10-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 | 0 ...inopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt10-5-9abe9cb11e3336a689a4116f8804c02a | 0 ...inopt10-6-bc7008e74f5eccab48d820877d83e7e4 | 0 ...inopt10-7-f4015c993efd5fc155e2faec784718d0 | 0 ...inopt10-8-c9624d1650d395f18d9e510cab4fab79 | 0 ...inopt11-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt11-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt11-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt11-3-a079ede4f4245e62b02f624efedfb597 | 0 ...inopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt11-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt11-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt11-7-9e7e6016590d33c617cb568cbd45ef68 | 0 ...inopt12-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt12-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt12-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt12-3-da45999e75a09b27161af2c7414c1170 | 0 ...inopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt12-5-e6db5d1ec4694475ab0b8f43eba415cd | 0 ...inopt12-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt12-7-8bbc680be8a68053db008789f335c2f0 | 0 ...inopt13-0-cafed8ca348b243372b9114910be1557 | 2 +- ...inopt13-1-16a1f74642d7ea4dac66a5ce15469c22 | 2 +- ...nopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 | 2 +- ...inopt13-2-114600d46ae86edcb66a500b4cac657d | 2 +- ...inopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...nopt13-4-c0f14def6a135cc50cba364e810ce28e} | 0 ...inopt13-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...nopt13-6-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...inopt13-8-3fda17e4414d191f837631438a19e700 | 0 ...nopt13-8-8eb53fb8f05a43ee377aa1c927857e7c} | 0 ...inopt14-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt14-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt14-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt14-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...inopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt14-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt14-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt14-7-c329b937ad1d7cf1c838640ef5f4d135 | 0 ...inopt14-8-3fda17e4414d191f837631438a19e700 | 0 ...inopt14-9-cdf19a17f3295447b66e6e6445742a74 | 0 ...inopt15-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt15-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...nopt15-10-7df9fe6075a96bf9849848c93e449469 | 0 ...nopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 | 0 ...inopt15-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt15-3-32fed3a53e7c15b549a71c0e71d93484 | 0 ...inopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 | 0 ...inopt15-5-dff9d122eb83760f08d1d77814c24c91 | 0 ...inopt15-6-717b85f496a5cf006cb352f9d884608d | 0 ...oinopt15-7-ba43a86694107dd4cb754d676935408 | 0 ...inopt15-8-7381c1f36c997afac91d8f8f29e865f3 | 0 ...inopt15-9-ccb54093d518eaca1e4644686e9e776e | 0 ...inopt16-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt16-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt16-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt16-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...inopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt16-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...inopt16-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt16-7-b3208400a48223a410b76a4bd1362da5 | 0 ...inopt17-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt17-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt17-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt17-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...inopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 | 0 ...inopt17-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt17-7-373b8a6363139ca37395b1cc8c9808d9 | 0 ...inopt18-0-cafed8ca348b243372b9114910be1557 | 2 +- ...inopt18-1-16a1f74642d7ea4dac66a5ce15469c22 | 2 +- ...nopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa | 2 +- ...inopt18-2-114600d46ae86edcb66a500b4cac657d | 2 +- ...inopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 | 0 ...nopt18-4-abf4b7f158999af331d5dbfddf32fa68} | 0 ...inopt18-8-4abc4f450a58ccdd0df2e345f1276979 | 0 ...nopt18-8-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...inopt19-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt19-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt19-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt19-3-6eda8acf464a18cfd9909255ddcef37e | 0 ...inopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt19-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt19-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt19-7-1e9c17669089eea559f8fa0b6977b249 | 0 ...oinopt2-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt2-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt2-3-52247b4dd98092bf829254e17424657d | 0 ...oinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt2-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt2-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 | 0 ...inopt20-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt20-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt20-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt20-3-4420d88e35aa84327bc95153eed299e0 | 0 ...inopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt20-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt20-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt20-7-e209254ae404366e6adca673d666aecb | 0 ...oinopt3-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt3-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt3-3-c64266431d312784ebc7b9ca07ab5188 | 0 ...oinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt3-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt3-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt3-7-e54bbab48fcc3c41806a101293577e9f | 0 ...oinopt4-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt4-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...oinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt4-5-744a018c78bae6e09853dd202981e850 | 0 ...oinopt4-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt4-7-c7e2ccaba57a581f27cfdcca76891133 | 0 ...oinopt5-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt5-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt5-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...oinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt5-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...oinopt5-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt5-7-f38878761b2aeeee0c04387dff60894d | 0 ...oinopt6-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt6-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt6-3-da45999e75a09b27161af2c7414c1170 | 0 ...oinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd | 0 ...oinopt6-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 | 0 ...oinopt7-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt7-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt7-3-c64266431d312784ebc7b9ca07ab5188 | 0 ...oinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt7-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt7-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt7-7-fa53198d9eecb9d274b09d4351b9274e | 0 ...oinopt7-8-3fda17e4414d191f837631438a19e700 | 0 ...joinopt7-9-b54d2a1f5d3bea81680ab06dead952c | 0 ...oinopt8-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt8-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...oinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt8-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt8-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt8-7-fa53198d9eecb9d274b09d4351b9274e | 0 ...oinopt8-8-3fda17e4414d191f837631438a19e700 | 0 ...oinopt8-9-9b26e8e33d3109e059e7494b53aee6fd | 0 ...oinopt9-0-cafed8ca348b243372b9114910be1557 | 2 +- ...oinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 | 2 +- ...inopt9-10-5c8be465ceef8151f172b82a13e81fa8 | 2 +- ...oinopt9-2-114600d46ae86edcb66a500b4cac657d | 2 +- ...oinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt9-4-c0f14def6a135cc50cba364e810ce28e} | 0 ...oinopt9-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt9-6-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...oinopt9-8-446c5e33062b109341add64a9860207d | 2 +- ...rtesian-0-e3c7f62795b0e7c14b41b0dc29c47619 | 2 +- ...join_1-10-b1114520feaa15ad7621b6a0e571c244 | 2 +- ...join_1-12-8fb6fea440e845ce23b06deed8f519fa | 2 +- ...join_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 | 2 +- ...join_1-16-c120e505c143721a36287bf992dbc1a1 | 2 +- ...join_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 | 2 +- ...join_1-20-d7979e9ab355d8856c2d12e66e7bb838 | 2 +- ...join_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b | 2 +- ...join_1-24-558e9ab6324f8082231b2fdd0e12f7ae | 2 +- ...pjoin_1-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_1-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_1-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_1-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...oin_10-3-68d65d622e45f86d4a6c7d1d09ef823b} | 0 ...join_10-3-bfb76fa2eccda1c64a85ea3841202705 | 0 ...oin_10-4-d31ad2289181131982ef3e9cd8c6386e} | 0 ...join_10-4-d4746bf376bce0bf561450c75b83fb74 | 0 ...join_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 | 0 ...oin_10-5-f7fb003fa65cadcd0b13cbdd7b355988} | 0 ...join_10-6-1094dbf800159e1e2382ec238b2466d7 | 0 ...oin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59} | 0 ...join_10-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_10-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_10-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_11-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_11-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_11-10-3d92573eecd22525a75464ad27b1dcaa | 0 ...oin_11-11-685ec4fbbf73330e026fba9b8cc53e92 | 1 - ...oin_11-12-c05c09243793df14546e8577ee369d58 | 1 - ...oin_11-13-1e0f21b664c1940d10531b3025be7f10 | 0 ...oin_11-14-f13be826d8f11be64c5414d591f70fd6 | 0 ...oin_11-15-b62714cb184523454314d19949dba9f4 | 0 ...oin_11-16-bee943a85cd82bd199b089fbdd7450af | 1 - ...join_11-2-b89ea2173180c8ae423d856f943e061f | 1 - ...join_11-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...join_11-4-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_11-5-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_11-6-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_11-7-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_11-8-beae7266b997c97798631b9dc558534f | 0 ...join_11-9-74936eafc274242beb49bc83d7a8af30 | 0 ...join_12-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_12-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_12-10-bc8140f238cfd13ea4fc4c4173a05454 | 0 ...oin_12-11-685ec4fbbf73330e026fba9b8cc53e92 | 1 - ...oin_12-12-c05c09243793df14546e8577ee369d58 | 1 - ...join_12-13-7e7645d5ee7d79991d8fdde072c8dbb | 0 ...oin_12-14-e9715c66355e9bc54155c79a4e82b34f | 0 ...oin_12-15-42b623410c408e09153a773db91c0334 | 0 ...oin_12-16-bee943a85cd82bd199b089fbdd7450af | 1 - ...oin_12-17-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...oin_12-18-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_12-19-b89ea2173180c8ae423d856f943e061f | 1 - ...join_12-2-b89ea2173180c8ae423d856f943e061f | 1 - ...oin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b | 0 ...oin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e | 0 ...oin_12-22-b1e1754efd667aa801b194985d41eb6e | 1 - ...join_12-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...join_12-4-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_12-5-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_12-6-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_12-7-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_12-8-3d016b9a5b0143b7a01b34472b569fb9 | 0 ...join_12-9-74936eafc274242beb49bc83d7a8af30 | 0 ...join_13-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_13-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_13-14-b92cb43f66838319f5d607c431fe1eb3 | 2 +- ...join_13-2-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_13-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_13-4-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_13-5-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_13-6-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_13-7-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_14-11-23d2ee09b01309b9cfcd0d625afc535d | 2 +- ...oin_14-13-95a09a0af2a77ac6b772c41a0f6a885a | 2 +- ...oin_14-15-e0f20915e07acf5ddfdbde40ef924e55 | 2 +- ...oin_14-17-5983d1e12e5a2bdd0f41549110e066ee | 2 +- ...oin_14-19-163af33279f8d08d747a00ffd1cdfac7 | 2 +- ...pjoin_14-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...oin_14-21-b6c3a3f68f212a966450286e23f59713 | 2 +- ...oin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 | 2 +- ...join_14-25-23f011143d8db18dd0f3d684adf7b8e | 2 +- ...oin_14-27-ba437062dd6661fc2fdcd41514711093 | 2 +- ...oin_14-29-d191c9ace84072ef430d7ed36ea8181b | 2 +- ...oin_14-31-fde2ad19052435bd8c155f18fe579070 | 2 +- ...oin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 | 2 +- ...join_14-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_14-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_14-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_15-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_15-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a | 2 +- ...oin_15-19-f49ef85423bb1766362f05651c9bb37f | 2 +- ...join_15-2-b89ea2173180c8ae423d856f943e061f | 2 +- ...oin_15-21-af3880637379684acd440830c2361f6e | 2 +- ...oin_15-23-e5e54dd17b69773812af376bfec33200 | 2 +- ...join_15-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_15-4-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_15-5-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_15-6-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_15-7-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_16-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_16-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_16-12-32f3716e22149e3d0c1f3ac26d414baf | 2 +- ...join_16-2-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_16-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_16-4-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_16-5-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_16-6-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_16-7-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_17-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_17-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_17-2-b89ea2173180c8ae423d856f943e061f | 2 +- ...oin_17-25-f066907fca3448b27aab623d05258a9a | 2 +- ...oin_17-26-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_17-27-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_17-29-65d3ae14a785f319658812b51e4620a3 | 2 +- ...join_17-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...oin_17-31-783fc1b07f117cd027395cf1c09149bc | 2 +- ...join_17-4-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_17-5-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_17-6-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_17-7-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_18-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_18-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...oin_18-11-fb15bd6eceb333302535c0bcdd2d078f | 0 ...oin_18-12-27762aa3d846e8f50d97350eaa7563a1 | 1 - ...oin_18-13-91f2af0da94e5a83601d02045980f556 | 1 - ...oin_18-14-1d0b88d6e6f84e485a05c712dd185531 | 1 - ...oin_18-15-21a6e09c24697160bf70cb17254ff504 | 1 - ...oin_18-16-35dfd037075aac5a9891bf99ea01e156 | 1 - ...oin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 - ...oin_18-18-ba77d1d4a1754489e2a8d5006b0af54d | 1 - ...oin_18-19-da6eb8ded1f72f518617339f58202cc5 | 1 - ...join_18-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...oin_18-20-940ff79d8f3b401a05b19b9417824d7a | 1 - ...oin_18-21-7ce17310f9fd362e2cc8a80211063264 | 1 - ...oin_18-22-a92f50cba6740abb3596c885a8157861 | 0 ...oin_18-23-f0675c53cddf75b012f64e797f5824c8 | 0 ...oin_18-24-a919b505c34a237ead34eea40b7d136c | 1 - ...join_18-25-d014ae435859316a1ad43548b72ecb7 | 1 - ...oin_18-26-52d3bbbbef7c580a324d991f52f44e1f | 1 - ...oin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 | 1 - ...oin_18-28-c83e75a3e18e68ef10d9970b3b8857ab | 1 - ...oin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 | 0 ...join_18-3-365488a703b0640acda73a7d7e6efa06 | 1 - ...oin_18-30-76e467313085467a3aa929b3665f9863 | 0 ...oin_18-31-7f1e9114f4db63982985068c4bf36a29 | 1 - ...oin_18-32-b59c406dae57fa6ab656a27e310ac54c | 1 - ...oin_18-33-95b42e61bf5eed67123e30d482e0c7fe | 1 - ...oin_18-34-14ae716c8c0f47db61b89a2a17e89415 | 1 - ...oin_18-35-549b3df453bf939e1c0a38df861245e5 | 1 - ...pjoin_18-4-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_18-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_18-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_18-7-c248759cecf0e2c223579f5e37f6669c | 0 ...join_18-8-724d37bd4a841f1fa4062f4f3e3eb353 | 0 ...join_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_19-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_19-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...oin_19-11-fb15bd6eceb333302535c0bcdd2d078f | 0 ...oin_19-12-27762aa3d846e8f50d97350eaa7563a1 | 1 - ...oin_19-13-4876f6e3c0ffde24afd82ec462962f19 | 1 - ...oin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 | 1 - ...oin_19-15-44ad799a82d847054f02d37139cc6aab | 1 - ...oin_19-16-8ee972ce0d73f28e98f46361082c83dc | 1 - ...oin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec | 1 - ...oin_19-18-b2546caccb501fca356930e6cd00ea2e | 1 - ...oin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 - ...join_19-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...oin_19-20-bceffa38b796fbc2a85daac23357da7b | 1 - ...oin_19-21-a1f4332461d1810334e7ae6d7d43f829 | 1 - ...oin_19-22-ebd323090f771227039cd21b1b8c4f3f | 1 - ...oin_19-23-90f0bad0606e63e4405eac9ec8c11deb | 1 - ...oin_19-24-3ae0277bb7a74cd6bee704313dc102e6 | 1 - ...oin_19-25-e621350131d50867015e75677cca031f | 1 - ...join_19-3-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_19-4-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_19-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_19-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_19-7-64b3fd0c215f4b8187866fa7eb55d34d | 0 ...join_19-8-ee04de92100063f9b2063baddb204864 | 0 ...join_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_2-10-1530d7565a22ace89ed16e0e1f9988ac | 2 +- ...join_2-12-48e7d8fcb2a4c31c5304045517a3bb89 | 2 +- ...join_2-14-3e7d435c7a5560abe539918adc4fa922 | 2 +- ...join_2-16-daeabb554f29b77f3c5ff7acff8c58ca | 2 +- ...join_2-18-411bc909c1c29811d184d1f4aceb25b3 | 2 +- ...join_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b | 2 +- ...pjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 | 2 +- ...join_2-24-709966d157a75ffb1c6ef70d7c72a498 | 2 +- ...pjoin_2-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_2-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_2-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_2-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_20-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_20-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 | 0 ...oin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 | 0 ...oin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 - ...oin_20-13-ba77d1d4a1754489e2a8d5006b0af54d | 1 - ...oin_20-14-da6eb8ded1f72f518617339f58202cc5 | 1 - ...oin_20-15-5acb0ec2e1abcc6d57de7529f414a75a | 0 ...oin_20-16-5b9583aecebb7480d778dc2a8605434a | 0 ...oin_20-17-e26f212ca34d303036299ba709f65522 | 0 ...oin_20-18-7fb8778a58cc27dc04d5947df15b250e | 1 - ...oin_20-19-6dd859e98c140df728f858a7a7598462 | 1 - ...join_20-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...oin_20-20-fca1f7361c63f0ba1e9d365cf0743845 | 1 - ...oin_20-21-28d04e54c0a986079ae9d52788846519 | 1 - ...oin_20-22-1baa93dd15ab33240255c5d6d5d57366 | 1 - ...oin_20-23-5c5eaf9922e7110c6d7260c738b17457 | 0 ...join_20-3-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_20-4-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_20-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_20-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...pjoin_20-7-c9981ec081cbf54c9323e0dee977934 | 0 ...join_20-8-5c9994d48509136efd1dcb4e3f3e7aff | 0 ...join_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_21-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_21-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_21-2-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_21-3-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_21-4-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_21-5-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_21-6-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_22-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_22-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_22-10-f0def0d77c93f6faebeca9b7a5340dbc | 0 ...oin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc | 0 ...oin_22-12-187c201f61c23833d0d193031926445a | 1 - ...oin_22-13-5c5c521954e5ade621fd0cbff5794c80 | 1 - ...join_22-14-b2534937758d2ff0a08e729f7b3ace4 | 1 - ...oin_22-15-83d9e3d23d6612a926e53d57a5d07307 | 1 - ...oin_22-16-abc95b36345d3e37abb196088897c7fe | 0 ...oin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...oin_22-18-2c29fc18b24368938f880a1bf3d5eb54 | 0 ...oin_22-19-5dc91a74bf4c43e20a8a213ad08c352e | 0 ...join_22-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...oin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad | 0 ...oin_22-21-f0def0d77c93f6faebeca9b7a5340dbc | 0 ...oin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc | 0 ...oin_22-23-187c201f61c23833d0d193031926445a | 1 - ...oin_22-24-5c5c521954e5ade621fd0cbff5794c80 | 1 - ...join_22-25-b2534937758d2ff0a08e729f7b3ace4 | 1 - ...oin_22-26-83d9e3d23d6612a926e53d57a5d07307 | 1 - ...join_22-3-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_22-4-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_22-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_22-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_22-7-8317b719ffcf85da8c618e8f2379a31b | 0 ...join_22-8-8cfa26d100b454c0b0f3443f62389abb | 0 ...join_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad | 0 ...join_25-0-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_25-1-365488a703b0640acda73a7d7e6efa06 | 2 +- ...oin_25-10-bd7036a4c0b57349a588b974ffaa502} | 0 ...oin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...in_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...oin_25-11-3af09654f8d38d21b5d26e6cc21210de | 0 ...join_25-12-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...in_25-12-6d835f651b099615df163be284e833de} | 0 ...oin_25-14-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ...join_25-15-442e69416faaea9309bb8c2a3eb73ef | 2 +- ...oin_25-16-beaea10543cfd753458b43d8aeb7571f | 2 +- ...oin_25-17-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_25-18-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 | 2 +- ...join_25-2-16367c381d4b189b3640c92511244bfe | 2 +- ...oin_25-20-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...oin_25-21-4d3e60a0e9bd8c12fdba4e010493537d | 2 +- ...oin_25-22-d0a93f40892e3894460553b443c77428 | 2 +- ...join_25-23-ae45f6382268c1035c11aa4fc8a23e2 | 2 +- ...oin_25-24-d5ad76f9178cb787cee037f25b19b270 | 2 +- ...join_25-3-bcde511b8d560ca83888265b96a781ff | 2 +- ...join_25-4-8067b7ab260021bc70af9ab47309ee35 | 2 +- ...join_25-5-545c0ea2ebd7a141526ee13059a857f6 | 2 +- ...join_25-6-dbcec232623048c7748b708123e18bf0 | 2 +- ...pjoin_3-10-efadb45f09b92d27233601394d53d79 | 2 +- ...join_3-12-77988b41de4b5a165c93942fbb2220d6 | 2 +- ...join_3-14-5456a3a43bfabcfdbb57257b633e299e | 2 +- ...join_3-16-fff1acc77150f6ea92fe8eefc04b079a | 2 +- ...join_3-18-96a658e934543dd77020ad264ec9b8c1 | 2 +- ...join_3-20-867845ed2cb38b55928f5310b4ae91bd | 2 +- ...join_3-22-5826f60da3a4b0c731c53340d50b0a06 | 2 +- ...join_3-24-c73176ea199d4d20934cf2659d42ec25 | 2 +- ...pjoin_3-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_3-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_3-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_3-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_4-14-a6efb153fe9be91c7055a5642e0f642d | 2 +- ...join_4-16-514034935af5348f67f1934cf5429d57 | 2 +- ...apjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 | 2 +- ...join_4-20-be9350a00a523e1758427a1c8e4bdf09 | 2 +- ...join_4-22-da023084bbab6727dc3823cfce500308 | 2 +- ...join_4-24-7783ab33a26d29a2c0235f52f0e4ad86 | 2 +- ...join_4-26-de14aa74d0da48dff6271410cc5dd98e | 2 +- ...pjoin_4-28-f71280b4b556515d068d074378c3a54 | 2 +- ...pjoin_4-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_4-30-a0d3abda611809460bcea4101967f21f | 2 +- ...join_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 | 2 +- ...pjoin_4-34-48869ba51ea36864e720f94b09bf6b3 | 2 +- ...join_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_4-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_4-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_4-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_5-14-e20bcd28cfd26259bcde9cffec8d2280 | 2 +- ...join_5-16-8a8dc418de3716f5112b9868f472e4b9 | 2 +- ...join_5-18-1f0dd26e6088c4f359fb691a8ef650bc | 2 +- ...join_5-20-18fe4e120d3c0d663c360b2504b4f88d | 2 +- ...join_5-22-6ed2620fe017e454459a83061f25951a | 2 +- ...join_5-24-87f758a20c15fa3b97b4ba6703ae541b | 2 +- ...join_5-26-1c1a9519e207edc3c3a927c986a37177 | 2 +- ...join_5-28-94440444fff7d2629a23a30fd778fcc7 | 2 +- ...pjoin_5-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_5-30-574d26a5179e5cebbbff5bbb425a9609 | 2 +- ...pjoin_5-32-17c99f827824994cfd21c40dbf4abfc | 2 +- ...join_5-34-4db871582cf4f3038d43d0a2d5ae6895 | 2 +- ...join_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_5-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_5-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_5-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...pjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...pjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...join_6-10-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_6-11-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_6-14-43123d2821871aa5b4d4a9e22e03d1ca | 2 +- ...join_6-16-449cb60e6537ba0810ea6879a7351a1e | 2 +- ...join_6-17-aa5f237005fb69b3f52808c80b4a276a | 2 +- ...apjoin_6-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_6-21-43123d2821871aa5b4d4a9e22e03d1ca | 2 +- ...join_6-23-449cb60e6537ba0810ea6879a7351a1e | 2 +- ...join_6-24-aa5f237005fb69b3f52808c80b4a276a | 2 +- ...pjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...pjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...join_7-11-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_7-12-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_7-13-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_7-16-18d2a689883588ae3c24b2810663ab80 | 2 +- ...join_7-19-ab46164010b27950c293f32fb89f9f8a | 2 +- ...apjoin_7-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_7-21-449cb60e6537ba0810ea6879a7351a1e | 2 +- ...join_7-22-aa5f237005fb69b3f52808c80b4a276a | 2 +- ...pjoin_7-23-1b6140f49d49eb1195c8551270b97ff | 2 +- ...pjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb | 0 ...pjoin_7-8-f983875c44b290b0884a22b6be6adc8} | 0 ...pjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f | 0 ...join_7-9-84a394d962965e38593883742cc32c0d} | 0 ...pjoin_8-0-43d53504df013e6b35f81811138a167a | 2 +- ...pjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_8-11-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_8-12-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_8-13-b89ea2173180c8ae423d856f943e061f | 2 +- ...pjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...apjoin_8-3-7cccbdffc32975f8935eeba14a28147 | 2 +- ...pjoin_8-5-a1f5562186e9e22e18ebd42208943525 | 0 ...join_8-5-eee18fc4192a4aa92a066eb66513be93} | 0 .../sort-1-10c8b0a592ed15eff488a71ec5909f45 | 2 +- ...desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 | 2 +- ..._desc_1-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._desc_1-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._desc_1-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_1-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...desc_2-11-47bc9cb5f049aaca33b394ea78578bdd | 2 +- ..._desc_2-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._desc_2-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._desc_2-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_2-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...desc_3-11-47bc9cb5f049aaca33b394ea78578bdd | 2 +- ..._desc_3-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._desc_3-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._desc_3-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_3-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...desc_4-11-47bc9cb5f049aaca33b394ea78578bdd | 2 +- ..._desc_4-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._desc_4-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._desc_4-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_4-9-b89ea2173180c8ae423d856f943e061f | 2 +- ..._desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ..._desc_5-1-365488a703b0640acda73a7d7e6efa06 | 2 +- ...desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a | 2 +- ..._desc_5-7-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ..._desc_5-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ..._desc_6-1-365488a703b0640acda73a7d7e6efa06 | 2 +- ...desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a | 2 +- ...n_desc_6-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ..._desc_6-8-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ..._desc_6-9-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...stats0-14-4f18f4b06db06844920b14e2d19471a9 | 10 +- ...stats0-15-4a7ed5b7e3deb4106204f8f950357e1c | 10 +- ...stats0-23-247568f4b3ce6b97979ca9d28c2ba05c | 6 +- ...stats0-30-4f18f4b06db06844920b14e2d19471a9 | 10 +- ...stats0-31-4a7ed5b7e3deb4106204f8f950357e1c | 10 +- .../stats0-7-247568f4b3ce6b97979ca9d28c2ba05c | 6 +- .../stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats2-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e | 0 ...stats2-11-912c785dbcef3251dd1d6b7fc56eae5b | 39 - .../stats2-2-a4fb8359a2179ec70777aad6366071b7 | 1 - .../stats2-3-16367c381d4b189b3640c92511244bfe | 1 - .../stats2-4-dc129f70e75cd575ce8c0de288884523 | 1 - .../stats2-5-6717e1026e24a38af19b6bce1951e3d7 | 0 .../stats2-6-45d4fb785fc94d266096fc29a9e90d73 | 0 .../stats2-7-6436db7a7506b560d0d1759db94a76b9 | 0 .../stats2-8-72621dba638b15d244850018e9f64d7 | 34 - .../stats2-9-6d93732dc2ca622eb60c171389caee8e | 19 - ...stats20-0-418ec894d08c33fd712eb358f579b7a0 | 1 - ...stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - ...stats20-2-4711e55648c25c86bb526ed50b0c3d09 | 1 - ...stats20-3-98c925a2b2c4de06e270e1b52437a98b | 0 ...stats20-4-82294461be4728b4b191414bf2fb3bd7 | 0 ...stats20-5-76509775cfe11bb98ee088188a07668a | 37 - ...stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 | 1 - ...stats20-7-82294461be4728b4b191414bf2fb3bd7 | 0 ...stats20-8-300c971de74642118d36d36349bc81aa | 37 - .../stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats3-1-418ec894d08c33fd712eb358f579b7a0 | 1 - ...stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 | 6 - ...stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa | 1 - .../stats3-12-892cb7ecc26e84f1c033b95a3ee3edc | 0 .../stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 | 0 ...stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa | 1 - ...stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 | 6 - .../stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 | 0 ...stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 | 0 ...stats3-18-eefbb3ee8b538aec85c609351e52901b | 38 - ...stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 | 0 .../stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 | 0 ...stats3-20-ca048ad81b3df7159822073d206f0790 | 0 .../stats3-3-ca048ad81b3df7159822073d206f0790 | 0 .../stats3-4-fa705a031ff5d97558f29c2b5b9de282 | 0 .../stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 | 43 - .../stats3-6-4bf1504274319c44d370b58092fe016c | 0 .../stats3-7-73d7d55d6e5a57aacce8618902904d | 32 - .../stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 | 0 .../stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 | 0 .../stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats4-1-418ec894d08c33fd712eb358f579b7a0 | 1 - ...stats4-10-a33b2c9d962e4921c98e62387f3989f7 | 0 ...stats4-12-30bc31441828a053d1a675b225a5d617 | 2 - ...stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 | 1000 ------- ...stats4-14-9c82167763a771c175c656786d545798 | 1000 ------- ...stats4-15-f02b95f20b526fcf2850b07ca6be4f8c | 36 - ...stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 | 36 - ...stats4-17-746b888d14c1b3fa28aa8549c174f6d9 | 36 - ...stats4-18-dbe13731de4ab2a3c23343b78525b2f7 | 36 - ...stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 | 39 - ...stats4-20-f63000f2c395b935199c9829964f98c1 | 39 - ...stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd | 0 ...stats4-22-f709d5c1d4c9f13b7a521867674ac54c | 0 .../stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd | 0 .../stats4-4-f709d5c1d4c9f13b7a521867674ac54c | 0 .../stats4-5-cd849c4fe1229428da98947e3e43b46d | 0 .../stats4-6-9c0d4354b6a9be351fa32a66ff58a177 | 0 .../stats4-7-16367c381d4b189b3640c92511244bfe | 1 - .../stats4-8-a4fb8359a2179ec70777aad6366071b7 | 1 - .../stats4-9-255ad4511130fb8c9ee9d65b7c95743f | 0 .../stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats5-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - .../stats5-2-6f5d06d6100de19ec398891cb2eae161 | 0 .../stats5-3-96d9aa9c32a081518604959dcfac42df | 19 - .../stats5-4-dbf81a12f6c19c14dce831e942870744 | 0 .../stats5-5-3980dfc2e4f882dd3bf478e56929361 | 33 - .../stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats6-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...stats6-10-653f748fa2d690502ba4fda407841a20 | 32 - ...stats6-11-1c9f833953403596ad50fd32e513642c | 32 - ...stats6-12-fdbe263d925f28d598a71b7a4c577492 | 39 - .../stats6-2-a4fb8359a2179ec70777aad6366071b7 | 1 - .../stats6-3-16367c381d4b189b3640c92511244bfe | 1 - .../stats6-4-a88c476a632cd92f748967fadb242405 | 0 .../stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats6-6-8926095434b70c83bf88c70559d38dce | 0 .../stats6-7-6615049191cfa4026a3a870c3c3749f4 | 0 .../stats6-8-e15e242124e61ede9196130cb3fb69e7 | 36 - .../stats6-9-e6b884de17a29eb476fd6cc502fc615d | 36 - .../stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats7-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...stats7-10-fdbe263d925f28d598a71b7a4c577492 | 38 - .../stats7-2-a4fb8359a2179ec70777aad6366071b7 | 1 - .../stats7-3-16367c381d4b189b3640c92511244bfe | 1 - .../stats7-4-a88c476a632cd92f748967fadb242405 | 0 .../stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats7-6-5d0c6aa78c9786d904728ff7adab85f2 | 18 - .../stats7-7-4912a8bbc138ab97ac0983bc90951de4 | 0 .../stats7-8-e15e242124e61ede9196130cb3fb69e7 | 36 - .../stats7-9-e6b884de17a29eb476fd6cc502fc615d | 36 - .../stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats8-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - .../stats8-10-ce78d029b7764edce3a26336cfac6a8 | 19 - ...stats8-11-6615049191cfa4026a3a870c3c3749f4 | 0 ...stats8-12-e6b884de17a29eb476fd6cc502fc615d | 36 - ...stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d | 19 - ...stats8-14-ea9afc1343991ed4d410231803a174f7 | 0 ...stats8-15-653f748fa2d690502ba4fda407841a20 | 36 - ...stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 | 19 - ...stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 | 0 ...stats8-18-1c9f833953403596ad50fd32e513642c | 36 - ...stats8-19-dbf72430cff3df0e6e9405ad64531b16 | 19 - .../stats8-2-a4fb8359a2179ec70777aad6366071b7 | 1 - ...stats8-20-bff592750d1057448d2cff7694b6dad2 | 0 ...stats8-21-e15e242124e61ede9196130cb3fb69e7 | 36 - ...stats8-22-e6b884de17a29eb476fd6cc502fc615d | 36 - ...stats8-23-653f748fa2d690502ba4fda407841a20 | 36 - ...stats8-24-1c9f833953403596ad50fd32e513642c | 36 - ...stats8-25-fdbe263d925f28d598a71b7a4c577492 | 39 - .../stats8-3-16367c381d4b189b3640c92511244bfe | 1 - .../stats8-4-a88c476a632cd92f748967fadb242405 | 0 .../stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats8-6-821e1f061960081b9b379d9bfb47f267 | 19 - .../stats8-7-8926095434b70c83bf88c70559d38dce | 0 .../stats8-8-e15e242124e61ede9196130cb3fb69e7 | 36 - .../stats8-9-fdbe263d925f28d598a71b7a4c577492 | 39 - .../stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats9-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...r_error_1-0-6d1832d28f897d0049de053617bd36 | 0 ...rror_1-0-6f3df708fa339df236ec9375d2ad37fa} | 0 ..._error_1-1-36eee5cbac5c0c3228e499805b32f6} | 0 ...error_1-1-887fe99770f53e7e0a0fbdc190118612 | 1 - ...rror_1-10-a31221a0c377c14e11b14484ddaa49a6 | 2 +- ...error_1-11-d58626190cded8d09f0457739a980eb | 2 +- ...rror_1-13-a31221a0c377c14e11b14484ddaa49a6 | 2 +- ...rror_1-14-60a231b64a4a0e414d0ddce1c813c614 | 2 +- ...rror_1-16-a31221a0c377c14e11b14484ddaa49a6 | 2 +- ..._error_1-2-2ca079278e0de95eecb5df315ce05c6 | 2 +- ...error_1-3-66e3e0c942759f679c270698b49bfcf1 | 2 +- ...error_1-4-d389db66cc7fd9b144445e364dac30e3 | 2 +- ...error_1-5-63abf47101c5097e66a9c3ee92b128e3 | 2 +- ...error_1-7-a31221a0c377c14e11b14484ddaa49a6 | 2 +- ...error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 | 2 +- ...yn_part-0-9505721cd28fc4ec94ef4baf07029027 | 0 ...yn_part-1-418ec894d08c33fd712eb358f579b7a0 | 1 - ...yn_part-2-c14f09f88961dbad4d800317079a9105 | 1 - ...yn_part-3-16367c381d4b189b3640c92511244bfe | 1 - ...yn_part-4-cc664530711607c530a2cd384e67a600 | 0 ...yn_part-5-76d56e06108f0c6da75aa821452fa873 | 0 ...rtition-1-418ec894d08c33fd712eb358f579b7a0 | 2 +- ...rtition-2-c14f09f88961dbad4d800317079a9105 | 2 +- ...rtition-4-aed016ae4b528521874a719a5b129a55 | 15 +- ...oscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - ...noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...scan_1-10-e6b884de17a29eb476fd6cc502fc615d | 36 - ...scan_1-11-653f748fa2d690502ba4fda407841a20 | 32 - ...scan_1-12-1c9f833953403596ad50fd32e513642c | 32 - ...scan_1-13-fdbe263d925f28d598a71b7a4c577492 | 38 - ...scan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab | 0 ...scan_1-15-db563e338e4f658e5072cc60aef15480 | 0 ...scan_1-16-e446db2c3ddce173d0a51bf77a489382 | 0 ...scan_1-17-82369b182db851e06bfddb62965e03a3 | 10 - ...scan_1-18-eece38f413a0a4f6f5b63cea0751d225 | 0 ...scan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e | 36 - ...oscan_1-2-a4fb8359a2179ec70777aad6366071b7 | 1 - ...scan_1-20-9871c619bb5bf0a28f8d60e6332a614f | 36 - ...scan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 | 32 - ...scan_1-22-521b49d223a32056025fb8dbd371a72a | 32 - ...scan_1-23-fba14d8647e0c8ca214b724486af7763 | 0 ...oscan_1-3-16367c381d4b189b3640c92511244bfe | 1 - ...oscan_1-4-7938a68104e163566da69ccc70a18f2c | 0 ...oscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 ...oscan_1-6-a1fd405e5175757aaa80033162c87670 | 10 - ...oscan_1-7-497861ae04753ffbb63ee43d89eedd9e | 0 ...oscan_1-8-623f3701ead20fff786c203d23dd60ae | 0 ...oscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 | 36 - ...tscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - ...rtscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...scan_1-10-c06f6ce9878b7eededf8c2a085ffb380 | 32 - ...scan_1-11-418ec894d08c33fd712eb358f579b7a0 | 1 - ...scan_1-12-3e423642a5a00dc66cc709f474a3ecef | 14 - ...scan_1-13-2fe3131322b6c82e217f27e95581e681 | 0 ...scan_1-14-2c66f128acea649e8debc3c4b04fcb9c | 36 - ...scan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 | 32 - ...scan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 | 0 ...tscan_1-2-a4fb8359a2179ec70777aad6366071b7 | 1 - ...tscan_1-3-16367c381d4b189b3640c92511244bfe | 1 - ...tscan_1-4-c95dc367df88c9e5cf77157f29ba2daf | 1 - ...tscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 | 1 - ...tscan_1-6-84967075baa3e56fff2a23f8ab9ba076 | 1 - ...tscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea | 1 - ...tscan_1-8-29279aa84d6ba9dea9e56b493504dd30 | 0 ...tscan_1-9-90d41ae72606f9616cb7b1303f997348 | 0 ...an_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - ...can_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...n_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 | 32 - ...n_1_23-11-418ec894d08c33fd712eb358f579b7a0 | 1 - ...n_1_23-12-3e423642a5a00dc66cc709f474a3ecef | 15 - ...n_1_23-13-2fe3131322b6c82e217f27e95581e681 | 0 ...n_1_23-14-2c66f128acea649e8debc3c4b04fcb9c | 36 - ...n_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 | 32 - ...n_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 | 0 ...an_1_23-2-a4fb8359a2179ec70777aad6366071b7 | 1 - ...an_1_23-3-16367c381d4b189b3640c92511244bfe | 1 - ...an_1_23-4-c95dc367df88c9e5cf77157f29ba2daf | 1 - ...an_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 | 1 - ...an_1_23-6-84967075baa3e56fff2a23f8ab9ba076 | 1 - ...an_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea | 1 - ...an_1_23-8-b158e24051ecb66b8af46743917771ca | 0 ...an_1_23-9-90d41ae72606f9616cb7b1303f997348 | 0 ..._error_1-1-36eee5cbac5c0c3228e499805b32f6} | 0 ...error_1-1-887fe99770f53e7e0a0fbdc190118612 | 1 - ..._to_map-0-aefd618b58ad6c37956755b6572cbc73 | 1 - ..._to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c | 2 - ...to_map-10-32997010bba305ec40812df254490730 | 0 ...to_map-11-d99f1b631bc42a6a57c581025230537a | 0 ...to_map-12-f793eb7b36a1d6379b90e241df62c72e | 3 - ...to_map-13-32997010bba305ec40812df254490730 | 0 ..._to_map-2-28d8e6677d025e1266ce95ae23dfd9ec | 0 ..._to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 | 3 - ..._to_map-4-f356516aec917fe0d34df4dc8d9d0f95 | 0 ..._to_map-5-269cfcefe2ea6c19ac6c824d3259bbda | 3 - ..._to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf | 0 ..._to_map-7-5641678c53ce6ef1dbce3994843cfcad | 3 - ..._to_map-8-84121d964faad3547f0e5cce9d268612 | 0 ..._to_map-9-903f3b3137bfa3f4594fec7b604c62bd | 3 - ...literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 | 2 +- .../subq2-1-235919a7ddb574662158503b8052e7ca | 2 +- ...as.attr-0-fc8183d758151be72b3d75d9df124504 | 2 +- ..._format-0-3b2fa9592648fc07c0d43e0d7d7f9411 | 0 ..._format-1-d498fb503b8f47db4741fdad3d266b4a | 0 ..._format-2-42119039bf8023f90b7f474f235c5dd5 | 1 - ..._format-3-77b57147024eb6b28cc9f525fdaab615 | 1 - ..._format-4-676cb274a770a6b9ca86df5dc7f912d4 | 0 ..._format-5-ef3052815ec41b5957627698ba06707b | 0 ..._format-6-891be0baec05e358a647dcca77724446 | 0 ..._format-7-208bcc9c918cbeb52907c8871be19cd5 | 0 ..._format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 | 0 ..._format-9-433d5dbbcf75ff68d6897f31baa46841 | 1 - ...le.attr-0-26c9d24eb6305ea2106c26bdca38012e | 2 +- ..._select-1-736d6a05e1fe3249a84eb58f7cd806d2 | 4 +- ..._select-3-35c08c648a66f09f2cf7cfa6019c2113 | 2 +- ...eclause-0-b38bf01368da26ec9c60e9433a9c59a1 | 0 ...eclause-1-3e38e42c5b72364c5461c626f312be8c | 0 ...eclause-2-183920d856ad75e6d1e15121d3cd7364 | 1 - ...tamp_1-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...stamp_1-0-d362501d0176855077e65f8faf067fa8 | 0 ...stamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 | 0 ...tamp_1-1-d362501d0176855077e65f8faf067fa8} | 0 ...amp_1-10-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-11-343c75daac6695917608c17db8bf473e} | 0 ...tamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 | 0 ...amp_1-12-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...amp_1-13-d242038c04dd4ee6075c7eebc0f75f17} | 0 ...amp_1-14-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-15-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-16-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-17-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-19-67f274bf16de625cf4e85af0c6185cac} | 0 ...tamp_1-2-1d7cf3a2512fa1876b422b79bbe05426} | 0 ...stamp_1-2-74f477a8b726f5193dd42ef378a793c4 | 0 ...amp_1-20-343c75daac6695917608c17db8bf473e} | 0 ...amp_1-21-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...tamp_1-21-d8fff1a6c464e50eb955babfafb0b98e | 0 ...amp_1-22-cdb04b49b836e0244f6f0857aea7da8a} | 0 ...amp_1-23-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-24-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-25-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-26-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-28-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-29-343c75daac6695917608c17db8bf473e} | 0 ...tamp_1-3-74f477a8b726f5193dd42ef378a793c4} | 0 ...stamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 | 0 ...tamp_1-30-273256141c33eb88194cad22eb940d21 | 0 ...amp_1-30-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...amp_1-31-9587976bd7e6caa5b667975c14e8dd53} | 0 ...amp_1-32-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-33-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-34-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-35-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-37-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-38-343c75daac6695917608c17db8bf473e} | 0 ...tamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d | 0 ...amp_1-39-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...tamp_1-4-d833b177fac3162215468dde991f71d1} | 0 ...amp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b} | 0 ...amp_1-41-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-42-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-43-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-44-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-46-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-47-343c75daac6695917608c17db8bf473e} | 0 ...tamp_1-48-7029255241de8e8b9710801319990044 | 0 ...amp_1-48-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...tamp_1-49-7a59f9f939efc4b96f8159d00b39ed3} | 0 ...tamp_1-5-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-50-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-51-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-52-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-53-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-55-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-56-343c75daac6695917608c17db8bf473e} | 0 ...amp_1-57-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...tamp_1-57-d362501d0176855077e65f8faf067fa8 | 0 ...amp_1-58-d362501d0176855077e65f8faf067fa8} | 0 ...tamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...tamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...tamp_1-8-e7b398d2a8107a42419c83771bda41e6} | 0 ...tamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...tamp_2-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...stamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 | 0 ...stamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 | 0 ...tamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028} | 0 ...tamp_2-10-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-11-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-12-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-12-7350308cbf49d6ebd6599d3802750acd | 0 ...amp_2-13-5f450162886ccc79af149541527f5643} | 0 ...amp_2-14-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-15-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-16-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-17-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-19-252aebfe7882335d31bfc53a8705b7a} | 0 ...tamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4} | 0 ...stamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 | 0 ...amp_2-20-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-21-240fce5f58794fa051824e8732c00c03} | 0 ...stamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b | 0 ...amp_2-22-469fe94fb60f4b00809190c303434641} | 0 ...amp_2-23-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-24-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-25-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-26-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-28-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-29-5181279a0bf8939fe46ddacae015dad8} | 0 ...tamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19} | 0 ...stamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 | 0 ...amp_2-30-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-30-ffe6b6ddaaba84152074f7781fba2243 | 0 ...amp_2-31-8f506498acf0c99c30960a00981ef460} | 0 ...amp_2-32-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-33-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-34-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-35-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-37-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-38-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-39-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-39-8236608f28681eac5503195096a34181 | 0 ...tamp_2-4-81d6d29dcb3fd12a519426dff64411d2} | 0 ...amp_2-40-972a007e54d1c09e9ac9549c19a32dbb} | 0 ...amp_2-41-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-42-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-43-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-44-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-46-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-47-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-48-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-48-654e5533ec6dc911996abc7e47af8ccb | 0 ...amp_2-49-650d2727b007638e0ed39b37c9498d66} | 0 ...tamp_2-5-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-50-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-51-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-52-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-53-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-55-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-56-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-57-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 | 0 ...amp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028} | 0 ...tamp_2-6-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...tamp_2-7-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...tamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...tamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...stamp_3-0-165256158e3db1ce19c3c9db3c8011d2 | 0 ...tamp_3-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...tamp_3-1-165256158e3db1ce19c3c9db3c8011d2} | 0 ...stamp_3-1-81edf5107270547641586aa02b4e7d9b | 0 ...amp_3-10-ffc79abb874323e165963aa39f460a9b} | 0 ...amp_3-11-7b1ec929239ee305ea9da46ebb990c67} | 0 ...tamp_3-12-165256158e3db1ce19c3c9db3c8011d2 | 0 ...amp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a} | 0 ...tamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 | 1 + ...amp_3-14-165256158e3db1ce19c3c9db3c8011d2} | 0 ...stamp_3-2-7a012a0d98729da25b5ac374855dcee4 | 0 ...tamp_3-2-81edf5107270547641586aa02b4e7d9b} | 0 ...stamp_3-3-6143888a940bfcac1133330764f5a31a | 0 ...tamp_3-3-7a012a0d98729da25b5ac374855dcee4} | 0 ...tamp_3-4-86514381187b246a5685577c1968c559} | 0 ...tamp_3-5-935d0d2492beab99bbbba26ba62a1db4} | 0 ...tamp_3-6-8fe348d5d9b9903a26eda32d308b8e41} | 0 ...tamp_3-7-6be5fe01c502cd24db32a3781c97a703} | 0 ...tamp_3-8-6066ba0451cd0fcfac4bea6376e72add} | 0 ...tamp_3-9-22e03daa775eab145d39ec0730953f7e} | 0 ...arison-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...arison-1-4b68f7ad0f8cf337e42bf16a45e15818} | 0 ...arison-2-60557e7bd2822c89fa8b076a9d0520fc} | 0 ...arison-3-f96a9d88327951bd93f672dc2463ecd4} | 0 ...arison-4-13e17ed811165196416f777cbc162592} | 0 ...arison-5-4fa8a36edbefde4427c2ab2cf30e6399} | 0 ...arison-6-7e4fb6e8ba01df422e4c67e06a0c8453} | 0 ...arison-7-8c8e73673a950f6b3d960b08fcea076f} | 0 ...arison-8-510c0a2a57dc5df8588bd13c4152f8bc} | 0 ...arison-9-659d5b1ae8200f13f265270e52a3dd65} | 0 ...p_lazy-2-bb5a4a13274290029bd07d95c2f92563} | 0 ...mp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 | 0 ...mp_null-2-51762cf5079877abf7d81127738f4e5} | 0 ...mp_null-2-b3071984dee42c9e698e947fcbc2a1e8 | 0 ...mp_udf-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...amp_udf-0-79914c5347620c6e62a8e0b9a95984af | 0 ...amp_udf-1-59fc1842a23369235d42ed040d45fb3d | 0 ...mp_udf-1-79914c5347620c6e62a8e0b9a95984af} | 0 ...p_udf-10-287614364eaa3fb82aad08c6b62cc938} | 0 ...p_udf-11-dbc23736a61d9482d13cacada02a7a09} | 0 ...p_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35} | 0 ...mp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6} | 0 ...mp_udf-14-6ab3f356deaf807e8accc37e1f4849a} | 0 ...p_udf-15-c745a1016461403526d44928a269c1de} | 0 ...p_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe} | 0 ...p_udf-17-b36e87e17ca24d82072220bff559c718} | 0 ...p_udf-18-dad44d2d4a421286e9da080271bd2639} | 0 ...mp_udf-19-79914c5347620c6e62a8e0b9a95984af | 0 ...p_udf-19-cb033ecad964a2623bc633ac1d3f752a} | 0 ...mp_udf-2-59fc1842a23369235d42ed040d45fb3d} | 0 ...amp_udf-2-9039f474f9a96e9f15ace528faeed923 | 0 ...mp_udf-20-59fc1842a23369235d42ed040d45fb3d | 0 ...p_udf-20-79914c5347620c6e62a8e0b9a95984af} | 0 ...p_udf-21-59fc1842a23369235d42ed040d45fb3d} | 0 ...mp_udf-3-9039f474f9a96e9f15ace528faeed923} | 0 ...amp_udf-3-b0fd4ca3b22eb732a32772399331352f | 0 ...amp_udf-4-80ce02ec84ee8abcb046367ca37279cc | 0 ...mp_udf-4-b0fd4ca3b22eb732a32772399331352f} | 0 ...amp_udf-5-66868a2b075de978784011e9955483d} | 0 ...amp_udf-6-1124399033bcadf3874fb48f593392d} | 0 ...mp_udf-7-5810193ce35d38c23f4fc4b4979d60a4} | 0 ...mp_udf-8-250e640a6a818f989f3f3280b00f64f9} | 0 ...mp_udf-9-975df43df015d86422965af456f87a94} | 0 ...ansform-0-d81d055660f6ef3d9cc60dd673a8c0fe | 2 +- ...nsform1-0-b6919fc48901e388c869c84ae0211102 | 0 ...nsform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 | 0 ...nsform1-2-25d6cab86c36d65fabf5645db3126a19 | 0 ...nsform1-3-8324a70d533a06a5171c1016b1fea7c3 | 0 ...nsform1-4-65527bae8e73262255ef83082c6968f9 | 0 ...nsform1-5-e0037a3f97ce0127a40d163af4c20ad5 | 0 ...ansform1-6-3b862abd732c9e9f0db50ad0b9dae6f | 1 - ...rm_ppr1-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...rm_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 | 2 +- ...rm_ppr2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...rm_ppr2-2-636c4938673a273299c8569295d27c99 | 2 +- ... clause-0-3b6afcbd622aa111ee260bebc763613d | 2 +- ... clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 | 2 +- ...cast_1-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d | 0 ...cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d} | 0 ..._cast_1-2-53a667981ad567b2ab977f67d65c5825 | 1 + ...dening-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...idening-0-630ac2c7e7dea4837384ccd572209229 | 0 ...dening-1-630ac2c7e7dea4837384ccd572209229} | 0 ...idening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 | 1 - ...idening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 | 0 ...dening-2-cfbdf2b6fca84c6e23d4e691d2221bd6} | 0 ...idening-3-65da8c67f6903286168acb39ac67fc04 | 1000 ------- ...dening-3-a0ba6952d9bf830d1d1ea7aebd3784a2} | 0 ...idening-4-65da8c67f6903286168acb39ac67fc04 | 1000 +++++++ ...ect_set-0-38512a3299e2390dd813e61a0f63f35e | 2 +- ...ect_set-1-78aa199d061d2ff9ba426849ea1eb449 | 2 +- ...ect_set-10-c8bc33095e1a195bb7b5e579d8d78db | 2 +- ...ect_set-11-5c3768074977ef68a1b9bb72eb9ef02 | 20 + ...ct_set-11-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...t_set-12-1d351f7e821fcaf66c6f7503e42fb291} | 0 ...ct_set-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...t_set-13-a00d1791b7fa7ac5a0505d95c3d12257} | 0 ...ect_set-13-c8bc33095e1a195bb7b5e579d8d78db | 20 - ...ect_set-14-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...t_set-15-863233ccd616401efb4bf83c4b9e3a52} | 0 ...t_set-16-a00d1791b7fa7ac5a0505d95c3d12257} | 0 ...ect_set-17-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...ect_set-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...lect_set-2-4747c35670a011344982573ba31a9bb | 1 + ...ect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 | 1 + ...ect_set-3-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...ct_set-4-1d351f7e821fcaf66c6f7503e42fb291} | 0 ...lect_set-4-c8bc33095e1a195bb7b5e579d8d78db | 20 - ...ect_set-5-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...ct_set-5-a7dc16cb82c595b18d4258a38a304b1e} | 0 ...ect_set-6-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...lect_set-6-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...ect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 | 20 + ...lect_set-7-c8bc33095e1a195bb7b5e579d8d78db | 20 - ...ect_set-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...ct_set-8-863233ccd616401efb4bf83c4b9e3a52} | 0 ...ect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...ct_set-9-a7dc16cb82c595b18d4258a38a304b1e} | 0 ...daf_corr-2-c6f2dc536bf105650a461816ae5e330 | 0 ...f_corr-2-e886f45c8f085596ffd420f89cdc2909} | 0 ...daf_corr-3-ddf417dbc3b551cc8df47d950cec03e | 2 +- ...af_corr-4-8771b2d83d14b3b641e8a77fcdc5081a | 2 +- ...af_corr-5-8abbd73784728b599f8c2a90f53da9fb | 2 +- ...af_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 | 2 +- ...af_corr-7-70e701f50c3418ff91649b2bd8287da3 | 2 +- ...af_corr-8-f2f0c7735f8b24266d5aaff96644e369 | 2 +- ...af_corr-9-e2a0fa75c43279764ebca015f62bcf16 | 2 +- ...ovar_pop-2-c6f2dc536bf105650a461816ae5e330 | 0 ...ar_pop-2-e886f45c8f085596ffd420f89cdc2909} | 0 ...var_pop-3-fe27ea6dae14250e8835baef3c2e20f6 | 2 +- ...var_pop-4-7af9231ae293d0c4b84050176e1d73ad | 2 +- ...var_pop-5-22004d36f6f3770db284644317770fcd | 2 +- ...var_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b | 2 +- ...var_pop-7-37e59e993e08216e6c69f88d6ac673ae | 2 +- ...var_pop-8-1e51388408dad651127edf940c11d91f | 2 +- ...var_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 | 2 +- ...var_samp-2-c6f2dc536bf105650a461816ae5e330 | 0 ...r_samp-2-e886f45c8f085596ffd420f89cdc2909} | 0 ...ar_samp-3-7aa25da7ccb88ba67b100888b6227aaf | 2 +- ...ar_samp-4-7e705a637950911e0a18059d8bf1fd2c | 2 +- ...var_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b | 2 +- ...ar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 | 2 +- ...ar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 | 2 +- ...ar_samp-8-44861ae58cf0951eeda28a5f778f778a | 2 +- ...ar_samp-9-234a5b02085d2228473d7ef15a6af683 | 2 +- ...numeric-0-86b9fb8bef8a5c51077623f1db3a0251 | 2 +- ...numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 | 2 +- ...numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 | 2 +- ...numeric-3-ff41f4450d6ae372633fde865ae187c6 | 2 +- ..._format-0-eff4ef3c207d14d5121368f294697964 | 0 ..._format-1-4a03c4328565c60ca99689239f07fb16 | 1 - ...prox_20-0-9ce9365f7b3f35a10b5305251c3e81ac | 0 ...prox_20-1-c7d32089880679d178dea94f1fe118e6 | 0 ...rox_20-10-b7e588217a3cd184dbbb8d419d3e33ae | 0 ...rox_20-11-5034ec132cb8b0a6bd6357a7e1abd755 | 0 ...rox_20-12-914ba18b45a27894bd82302f07efc789 | 0 ...rox_20-13-4bd5703fa32f3283f38841acadc97adb | 0 ...rox_20-14-d861a06b90896a097901d64ab9fbec53 | 0 ...rox_20-15-ca796efecd0d064e9e688a17ce75d80f | 0 ...rox_20-16-c838e13d9aafe1212a76d2cf5fe085a0 | 0 ...rox_20-17-b89ea2173180c8ae423d856f943e061f | 1 - ...rox_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 - ...prox_20-19-b931aec4add0a66c23e444cdd5c33c5 | 0 ...prox_20-2-ac53a7ba5e8a208255008d3a71fa321a | 0 ...rox_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 - ...rox_20-21-3cd4e1282d82d07785051a1cf0e9b4ff | 1 - ...rox_20-22-ed1aec1a908310db90c5f8667631a1df | 1 - ...rox_20-23-333d72e8bce6d11a35fc7a30418f225b | 1 - ...rox_20-24-61903781f5cd75e6f11d85e7e89c1cb3 | 1 - ...rox_20-25-15f40568b41c4505841f5ad13c526f51 | 1 - ...rox_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 - ...rox_20-27-ee4c065e8557632a9ee348dd9223c3a1 | 1 - ...rox_20-28-ba77d1a26f87385f046129b6eb7d2ec3 | 1 - ...rox_20-29-956d6b6bc69c8035f80de2e60eda65fb | 1 - ...prox_20-3-1dae5b2a11507c83b0f76e677a368712 | 0 ...rox_20-30-d196cc7f52bb6ae19a5e66eb2a99577c | 1 - ...rox_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 - ...rox_20-32-dbcec232623048c7748b708123e18bf0 | 1 - ...rox_20-33-f28c7b0408737da815493741c806ff80 | 0 ...rox_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 - ...rox_20-35-3cd4e1282d82d07785051a1cf0e9b4ff | 1 - ...rox_20-36-ed1aec1a908310db90c5f8667631a1df | 1 - ...rox_20-37-333d72e8bce6d11a35fc7a30418f225b | 1 - ...rox_20-38-61903781f5cd75e6f11d85e7e89c1cb3 | 1 - ...rox_20-39-15f40568b41c4505841f5ad13c526f51 | 1 - ...prox_20-4-b2e21ffa55342d4f3c243728dfe6b11f | 0 ...rox_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 - ...rox_20-41-ee4c065e8557632a9ee348dd9223c3a1 | 1 - ...rox_20-42-ba77d1a26f87385f046129b6eb7d2ec3 | 1 - ...rox_20-43-956d6b6bc69c8035f80de2e60eda65fb | 1 - ...rox_20-44-d196cc7f52bb6ae19a5e66eb2a99577c | 1 - ...rox_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 - ...prox_20-5-8ae1465266d28bc2e5da8d89617873c4 | 0 ...prox_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 | 0 ...prox_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 | 0 ...pprox_20-8-7e3cf228c457279965b7414bd05527f | 0 ...prox_20-9-5aea8aa95a85c46284f7c1f45978a228 | 0 ...prox_23-0-477a942be95c0616c72f02a0077f9ace | 0 ...prox_23-1-c7d32089880679d178dea94f1fe118e6 | 0 ...rox_23-10-b7e588217a3cd184dbbb8d419d3e33ae | 0 ...rox_23-11-5034ec132cb8b0a6bd6357a7e1abd755 | 0 ...rox_23-12-914ba18b45a27894bd82302f07efc789 | 0 ...rox_23-13-4bd5703fa32f3283f38841acadc97adb | 0 ...rox_23-14-d861a06b90896a097901d64ab9fbec53 | 0 ...rox_23-15-ca796efecd0d064e9e688a17ce75d80f | 0 ...rox_23-16-c838e13d9aafe1212a76d2cf5fe085a0 | 0 ...rox_23-17-b89ea2173180c8ae423d856f943e061f | 1 - ...rox_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 - ...prox_23-19-b931aec4add0a66c23e444cdd5c33c5 | 0 ...prox_23-2-ac53a7ba5e8a208255008d3a71fa321a | 0 ...rox_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 - ...rox_23-21-3cd4e1282d82d07785051a1cf0e9b4ff | 1 - ...rox_23-22-ed1aec1a908310db90c5f8667631a1df | 1 - ...rox_23-23-333d72e8bce6d11a35fc7a30418f225b | 1 - ...rox_23-24-61903781f5cd75e6f11d85e7e89c1cb3 | 1 - ...rox_23-25-15f40568b41c4505841f5ad13c526f51 | 1 - ...rox_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 - ...rox_23-27-ee4c065e8557632a9ee348dd9223c3a1 | 1 - ...rox_23-28-ba77d1a26f87385f046129b6eb7d2ec3 | 1 - ...rox_23-29-956d6b6bc69c8035f80de2e60eda65fb | 1 - ...prox_23-3-1dae5b2a11507c83b0f76e677a368712 | 0 ...rox_23-30-d196cc7f52bb6ae19a5e66eb2a99577c | 1 - ...rox_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 - ...rox_23-32-dbcec232623048c7748b708123e18bf0 | 1 - ...rox_23-33-f28c7b0408737da815493741c806ff80 | 0 ...rox_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 - ...rox_23-35-3cd4e1282d82d07785051a1cf0e9b4ff | 1 - ...rox_23-36-ed1aec1a908310db90c5f8667631a1df | 1 - ...rox_23-37-333d72e8bce6d11a35fc7a30418f225b | 1 - ...rox_23-38-61903781f5cd75e6f11d85e7e89c1cb3 | 1 - ...rox_23-39-15f40568b41c4505841f5ad13c526f51 | 1 - ...prox_23-4-b2e21ffa55342d4f3c243728dfe6b11f | 0 ...rox_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 - ...rox_23-41-ee4c065e8557632a9ee348dd9223c3a1 | 1 - ...rox_23-42-ba77d1a26f87385f046129b6eb7d2ec3 | 1 - ...rox_23-43-956d6b6bc69c8035f80de2e60eda65fb | 1 - ...rox_23-44-d196cc7f52bb6ae19a5e66eb2a99577c | 1 - ...rox_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 - ...prox_23-5-8ae1465266d28bc2e5da8d89617873c4 | 0 ...prox_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 | 0 ...prox_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 | 0 ...pprox_23-8-7e3cf228c457279965b7414bd05527f | 0 ...prox_23-9-5aea8aa95a85c46284f7c1f45978a228 | 0 .../udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 | 0 .../udf1-1-8281592c818ada269024ac669bec78da | 0 .../udf1-2-c7648c913ec336559fb67e3ab6938c8f | 0 .../udf1-3-adc1ec67836b26b60d8547c4996bfd8f | 1 - .../udf2-3-c5938fcdd5675b58a4ed54269b5f5591 | 2 +- .../udf3-0-66a2b926dd5439d4da9eb58743c18a8c | 0 .../udf3-1-1d04874d496d05cfe0b9d86de1111 | 0 .../udf3-2-25fe77d053e2bad8ae99757ce237052e | 0 .../udf3-3-adc1ec67836b26b60d8547c4996bfd8f | 1 - .../udf6-3-e579646b969eef49b09656114da52a73 | 2 +- .../udf6-5-fe336cd9850d6357980bd19139f76e | 2 +- .../udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 | 2 +- .../udf8-4-9f22d5a65353432826a526b1d76eb65b | 2 +- .../udf9-1-dd0981dc44ac24d445af5412e9f7fa8c | 2 +- .../udf_E-0-33251f00f840de3672f19d353fcfa66f | 0 ... udf_E-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ... udf_E-1-cad0779d18f326c8e453bf2b5fe43596} | 0 ... udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02} | 0 ... udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a} | 0 .../udf_E-4-33251f00f840de3672f19d353fcfa66f | 0 ... udf_E-4-e8924af3bf99d2e01546a965303ffd09} | 0 ... udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4} | 0 ... udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02} | 0 ... udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a} | 0 ... udf_E-8-e8924af3bf99d2e01546a965303ffd09} | 0 ...udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 .../udf_PI-0-b28e761e5564b51f98f182f561c1369f | 0 ...udf_PI-1-13fd9345fd15b654d18b2707e5274b2b} | 0 ...udf_PI-2-97a12f6967726e425469ecfa70177ff0} | 0 ...udf_PI-3-9c1476a2eab7455594e97b338ee3c188} | 0 ...udf_PI-4-890f3c276eff2c459d8dc79d5a71c866} | 0 .../udf_PI-4-b28e761e5564b51f98f182f561c1369f | 0 ...udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d} | 0 ...udf_PI-6-97a12f6967726e425469ecfa70177ff0} | 0 ...udf_PI-7-9c1476a2eab7455594e97b338ee3c188} | 0 ...udf_PI-8-890f3c276eff2c459d8dc79d5a71c866} | 0 ...df_abs-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 | 1 - ...udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 | 1 + ...udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 | 6 - ...udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 | 0 ...udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 | 6 + ...df_abs-3-50cb3c23902cd29e0dbff188c71062e5} | 0 ...udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 | 1 - .../udf_abs-4-30cd5a94c13e1619ee18b9551db879c | 0 ...udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 | 1 + ...udf_abs-5-343e899acb67c283391387f02aa7b5c4 | 1 - ...df_abs-5-5cd4c198e0de884ad436864b95fece6c} | 0 ...udf_abs-6-7aa32a019499c6464aded2e357c6843b | 1 + ...df_acos-0-4f49cb5a5c87efea534d63ed76435f06 | 1 - ...f_acos-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_acos-1-4f49cb5a5c87efea534d63ed76435f06 | 1 + ...df_acos-1-d330511cf4f626cd844b18f57f99a85f | 6 - ...df_acos-2-86fca49baf270873b46709c9eaeab87b | 1 - ...df_acos-2-d330511cf4f626cd844b18f57f99a85f | 6 + ...f_acos-3-661a0a85283df2a5c1567d60850e362b} | 0 ...df_acos-3-f7f199e5f3dde8056465d55aca29e884 | 1 - ...df_acos-4-23d588eece08fbea7431044524f1cecf | 1 + ...df_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a | 1 - ...df_acos-5-578612589fdb1ae21ee488924848fb4e | 1 + ...df_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e | 1 - ...df_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 | 1 + ...udf_add-0-5db764318a918a5f6d7c1d95d9e86045 | 2 +- ...udf_add-1-400b238f4e6cdf7120be566b0ef079c5 | 2 +- ..._array-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._array-1-2e9c2a7d9325bd9a53cf9f181b6333ee} | 0 ..._array-2-570741914bb78300b0233e5f38d7f08a} | 0 ...f_array-2-db6d3c6c67faed3ceb019cb1993502f6 | 0 ..._array-3-47818d42e5e7667d8754c3f9a4b8053a} | 0 ..._array-4-51410e4d4d679fe5a8dd7a860f4efc47} | 0 ...ntains-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...ontains-0-d9a90108b052b111e8de4433e008b25a | 1 - ...ontains-1-d9a90108b052b111e8de4433e008b25a | 1 + ...ontains-1-eff16c7836252e01f3d8190cd833f79c | 4 - ...ontains-2-42d966b28e61a465d638bffc20ac7247 | 1 - ...ontains-2-eff16c7836252e01f3d8190cd833f79c | 4 + ...ontains-3-6b700cb69af298aef45b96bf5ac862d} | 0 ...ontains-3-ec0cd851fd8135dd9bc822d9a0432569 | 1 - ...ntains-4-bdb5a7e0ab81172a438145a1c406b1e8} | 0 ..._ascii-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_ascii-0-72924c23459330cca6a54c70b12a542c | 1 - ...f_ascii-1-72924c23459330cca6a54c70b12a542c | 1 + ...f_ascii-1-dab8656d7f001e85615442d60df4b6b3 | 6 - ...f_ascii-2-a9e207503f52d43903877fb998eabeaa | 0 ...f_ascii-2-dab8656d7f001e85615442d60df4b6b3 | 6 + ...f_ascii-3-28fc6497c9835c2ef331aba44576f1b1 | 1 - ..._ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34} | 0 ...udf_ascii-4-db9a06881a216f0252fa786d98c9bf | 1 + ...f_asin-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_asin-0-99da197a53767060e3fa0250254d59cd | 1 - ...df_asin-1-3d0c3fa6121f8f5158d221074f1d4129 | 6 - ...df_asin-1-99da197a53767060e3fa0250254d59cd | 1 + ...df_asin-2-3d0c3fa6121f8f5158d221074f1d4129 | 6 + ...df_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f | 1 - ...f_asin-3-4b7ee6310a49ebf784a4a712748348ac} | 0 ...df_asin-3-b67069d37df3a7fb7a3e2138d8558343 | 1 - ...df_asin-4-929dabad86ef0e564802de8f663a9e66 | 1 - ...df_asin-4-a3edf78ff8ccc629ee7d7518707b69ce | 1 + ...df_asin-5-1ee8715cce9831623d0af0031964d284 | 1 - ...df_asin-5-8dcbcf784496053e3b57c579aca809a6 | 1 + ...df_asin-6-114c8141f1e831c70d70c570f0ae778f | 1 + ...f_atan-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_atan-0-c79ed30c2444c8493d0db98c33c9132b | 1 - ...df_atan-1-77e7ac1b89a4eac9102176cd73f67a62 | 4 - ...df_atan-1-c79ed30c2444c8493d0db98c33c9132b | 1 + ...df_atan-2-77e7ac1b89a4eac9102176cd73f67a62 | 4 + ...df_atan-2-bf1c7875364d5518e0ac9c1ac6943764 | 1 - ...df_atan-3-9a6252f36fe5ec1387186bf47d74a139 | 1 - ...f_atan-3-e6f97a834028a67e6c3033949f98fbf8} | 0 ...df_atan-4-c79ed30c2444c8493d0db98c33c9132b | 1 - ...df_atan-4-eacd47571ba5c67f11e025d8d4de5811 | 1 + ...df_atan-5-77e7ac1b89a4eac9102176cd73f67a62 | 4 - ...df_atan-5-c79ed30c2444c8493d0db98c33c9132b | 1 + ...df_atan-6-77e7ac1b89a4eac9102176cd73f67a62 | 4 + ...df_atan-6-bf1c7875364d5518e0ac9c1ac6943764 | 1 - ...df_atan-7-9a6252f36fe5ec1387186bf47d74a139 | 1 - ...f_atan-7-e6f97a834028a67e6c3033949f98fbf8} | 0 ...df_atan-8-eacd47571ba5c67f11e025d8d4de5811 | 1 + ...udf_avg-0-2d715528b290951fb9874f60d7e9b537 | 2 +- ...udf_avg-1-c707c56871a903e4e022b3df5c92fc3f | 2 +- ..._bigint-0-6c5b1e4b9d725caeb786bb18448a7927 | 2 +- ..._bigint-1-4636e4f0083ea54814995a03b7c81202 | 2 +- ...df_bin-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b | 1 - ...udf_bin-1-843803a1b6ada107c11397af1a2f55d6 | 5 - ...udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b | 1 + ...udf_bin-2-5ee3932ab9cd164f1005a4413a68007b | 1 - ...udf_bin-2-843803a1b6ada107c11397af1a2f55d6 | 5 + .../udf_bin-3-6fda27c8567ac896538cba3f2b230ab | 1 + .../udf_bin-3-b72fc578a7c677e15b8598248c81901 | 1 - ...udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 | 1 + ...ap_and-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...map_and-0-abea2a2780fad42422774174cbfd603d | 1 - ...map_and-1-414291f11711df40fb8362e0a0156b25 | 1 - ...map_and-1-c9e0d8424ec5f433565397b113ae4f57 | 1 + ...map_and-2-5a60dfc80bef392fa390adddab0c0f87 | 1 + ...map_and-2-d550d017f9eb9176593719ea34eaae9b | 0 ...map_and-3-a486db1f5a06f9623a0e5abd7737b0c6 | 0 ...ap_and-3-d550d017f9eb9176593719ea34eaae9b} | 0 ...map_and-4-6320e1c4829024f819d3883df9a782c0 | 0 ...ap_and-4-a486db1f5a06f9623a0e5abd7737b0c6} | 0 ...ap_and-5-549dbeb1293c4c49ae08bf08acdbdf23} | 0 ...map_and-5-ff2860a163cbe78d5affac8047199296 | 10 - ...map_and-6-d550d017f9eb9176593719ea34eaae9b | 0 ...map_and-6-ff2860a163cbe78d5affac8047199296 | 10 + ...ap_and-7-d550d017f9eb9176593719ea34eaae9b} | 0 ..._empty-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...p_empty-0-6c80525a4eab6761596e6ad5aa75bc90 | 1 - ...p_empty-1-a03987655a167f5b18c37b643391a0df | 1 - ..._empty-1-a174269b5d1757398ab7f89cf1c97bfa} | 0 ..._empty-2-f5d9880a3278b5632b356bbe6ecd90d3} | 0 ...tmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c | 1 - ...map_or-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...tmap_or-1-85cab84fba419b436b033e3ea07e02ef | 1 - ...tmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef | 1 + ...tmap_or-2-ab062e08acfd7e552a64ea967a0360c8 | 1 + ...tmap_or-2-d550d017f9eb9176593719ea34eaae9b | 0 ...tmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 | 0 ...map_or-3-d550d017f9eb9176593719ea34eaae9b} | 0 ...tmap_or-4-6320e1c4829024f819d3883df9a782c0 | 0 ...map_or-4-a486db1f5a06f9623a0e5abd7737b0c6} | 0 ...map_or-5-549dbeb1293c4c49ae08bf08acdbdf23} | 0 ...tmap_or-5-ea92fff4d814552b57535ed342a5dde0 | 10 - ...tmap_or-6-d550d017f9eb9176593719ea34eaae9b | 0 ...tmap_or-6-ea92fff4d814552b57535ed342a5dde0 | 10 + ...map_or-7-d550d017f9eb9176593719ea34eaae9b} | 0 ...ise_and-0-e2374700cd32add926992d5539bd463a | 2 +- ...ise_and-1-2e63ac31262106160ab043027e356a4b | 2 +- ...ise_not-0-34abab2f47f90f0f194ef44aed1cdd7f | 2 +- ...ise_not-1-ccc0c59ea3e29b6d55e1edee7029155d | 2 +- ...wise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 | 2 +- ...wise_or-1-272722c23fece2807c08191d3969c3bb | 2 +- ...ise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 | 2 +- ...ise_xor-1-8fc9b9bf1aced42754419310784e0a9f | 2 +- ...boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 | 2 +- ...boolean-1-23178b473a9d25d6be0abe378d44fb0e | 2 +- ...f_case-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_case-0-5bcbe4c0e0a75462160814a10b9449f4 | 1 - ...df_case-1-54acf006155d8822a50e47729be24004 | 1 - ...df_case-1-5bcbe4c0e0a75462160814a10b9449f4 | 1 + ...df_case-2-54acf006155d8822a50e47729be24004 | 1 + ...df_case-2-98ee676f92950375917f09d2e492253f | 0 ...f_case-3-48789112d79aeb450d9f49184fc20e1c} | 0 ...df_case-3-ec7343402fd77807842a0eaf2497a47c | 1 - ...df_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 | 1 - ...df_case-4-d39ed6ecd256fa99657f13709cb1c6e3 | 1 + ...df_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb | 1 + ...df_case-6-ff583116ba2edd78202349faf1e757dc | 1 + ...pruning-0-dd2d7a075df235f17c26bac8713e939c | 0 ..._thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 | 0 ..._thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 | 3 - ...df_ceil-0-4b40e67b8ca75729ab07df966d814e06 | 2 +- ...df_ceil-1-f410065d893a263f375fcf74072877bb | 2 +- ...ceiling-0-d5685e38400e68341e36a59671dcbdfd | 2 +- ...ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df | 2 +- ...oalesce-0-8d1c97c292c51447e58606f2cefa87c1 | 1 - ...oalesce-1-e2c086f5148f10264c09ecbd7284c809 | 4 - ...coalesce-2-bd78a25868e5598ea275e0be5e4c716 | 0 ...oalesce-3-badb02b0dfa13d6a1519e6198bb511d2 | 1 - ...oalesce-4-83f323874d7941c463ced2aee6cc5157 | 0 ...oalesce-5-4bcad31a47f4bfc3baef75b65baa8082 | 11 - ..._string-0-32b16ab99287db115e8de5214ac24b77 | 10 - ...concat-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_concat-0-7bc53505a4e6587132870d8d0a704d2 | 1 - ..._concat-1-765c520f239cdff1ea3f8d22ba83e031 | 5 - ...f_concat-1-7bc53505a4e6587132870d8d0a704d2 | 1 + ..._concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 | 1 - ..._concat-2-765c520f239cdff1ea3f8d22ba83e031 | 5 + ..._concat-3-a38183c2685e912befe6246f1b6f93b8 | 1 + ..._concat-4-a20ebbc181e5ee4a1c22ddafd212ddde | 1 + ...insert1-2-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...insert2-2-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ncat_ws-0-4c8bd7794a527e544c838408313eeaa8 | 1 - ...cat_ws-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...ncat_ws-1-4c8bd7794a527e544c838408313eeaa8 | 1 + ...ncat_ws-1-b8b80f7e9bf4348367444c73020b3cab | 4 - ...ncat_ws-2-b8b80f7e9bf4348367444c73020b3cab | 4 + ...ncat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c | 0 ...ncat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 | 0 ...cat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c} | 0 ...ncat_ws-4-a507af4996b13433b0ae100fcb32358f | 0 ...cat_ws-4-b13a1f7f63d2a54efa331c82bd635d63} | 0 ...cat_ws-5-a507af4996b13433b0ae100fcb32358f} | 0 ...ncat_ws-5-ca4f051369183cae36fc9a7bec6a9341 | 1 - ...ncat_ws-6-98276439c0605401ff89c6a5ae54be09 | 0 ...ncat_ws-6-ca4f051369183cae36fc9a7bec6a9341 | 1 + ...ncat_ws-7-8f08128276e7e50eeb5a6932c763126c | 1 - ...cat_ws-7-97071809ba7701b78e3729996f14b591} | 0 ...ncat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 | 1 + ...ncat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 | 1 - ...ncat_ws-9-8f60d81b410f4825809aa510806f2df2 | 1 + ...f_conv-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_conv-0-d552befca345f3396464529cfde9f75a | 1 - ...df_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab | 7 - ...df_conv-1-d552befca345f3396464529cfde9f75a | 1 + ...f_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 | 3 - ...f_conv-10-f9ea15984e84250494e81e25d6a401c0 | 1 + ...f_conv-11-2ce9111b47ed100bebc8d08de54efd1f | 3 + ...df_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab | 7 + ...df_conv-2-6d61a2118b54843716aef87fe539b595 | 1 - ...df_conv-3-5f43d0bec0421c86c49e2546c5ee923a | 1 + ...df_conv-3-97161f7a60851d445b23c4ebe4095a1d | 1 - ...df_conv-4-568e843076f358c404a8634b18541c55 | 1 - ...df_conv-4-5df8d45902a3537e67545e629a96328a | 1 + ...df_conv-5-3f23d98799b825a2e9594066f973d183 | 1 - ...df_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 | 1 + ...df_conv-6-4981b5b92d87fd1000fa0ac26fa6163a | 1 - ...df_conv-6-e5430adfa782ea9094e570d339626c0f | 1 + ...df_conv-7-687f9b8a09f458d771d5641eec40031b | 1 + ...df_conv-7-77bd25ad13e3697c80464e4a2682360e | 1 - ...df_conv-8-2fae52d392251be476e0c8f6071a4aeb | 1 - ...udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 | 1 + ...df_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 | 1 + ...df_conv-9-2f0098c00c10044522cd23a4a2f54957 | 1 - ...udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 | 1 - ...df_cos-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 | 4 - ...udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 | 1 + ...udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 | 4 + ...udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 | 1 - ...df_cos-3-166acc86afd6ececfe43800e38f106c9} | 0 ...udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 | 1 - ...udf_cos-4-efda2f85872c6144280970eab84ef4d4 | 1 + ...f_count-0-534a9b25b87d09e418645b1216949560 | 2 +- ...f_count-1-d566feb21bc894b97e6416b65fe5c02f | 2 +- ..._count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e | 2 +- ..._count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e | 2 +- ...f_count-13-9286bc5f08bf4db183719b1b49dc5b7 | 2 +- ...f_count-3-e43165f41597d2a1c9e8cf780b99a4a8 | 2 +- ...f_count-5-bdee61c35a27bfab974e2ba199d5dfa4 | 2 +- ...f_count-7-b975ad0d5f293508ce4832a7b19399b6 | 2 +- ...f_count-9-75b3d8a0dac332ea00af5ef8971ca643 | 2 +- ...ate_add-0-74d34471bfa0880230d8e3351eb0ab45 | 2 +- ...date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 | 2 +- ...ate_sub-0-f8fbf85026da1b0778fd325d9b5dae33 | 2 +- ...ate_sub-1-7efeb74367835ade71e5e42b22f8ced4 | 2 +- ...atediff-0-3bd040a96a4568e7ea4922faa408ada5 | 2 +- ...atediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 | 2 +- ...udf_day-0-c4c503756384ff1220222d84fd25e756 | 2 +- .../udf_day-1-87168babe1110fe4c38269843414ca4 | 2 +- ...ofmonth-0-7b2caf942528656555cf19c261a18502 | 2 +- ...ofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 | 2 +- ...egrees-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...degrees-0-85f4957cd7cd6c517f6941af1289d8ae | 0 ...egrees-1-f24ce67606944e23a4adc79f91cf0c17} | 0 ...egrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4} | 0 ...egrees-3-2fd3a55901fe765f8f154531a7f5fd6b} | 0 ...egrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96} | 0 ...degrees-4-85f4957cd7cd6c517f6941af1289d8ae | 0 ...egrees-5-3a6468b02be2605c91b31987e76fb9a8} | 0 ...egrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4} | 0 ...egrees-7-2fd3a55901fe765f8f154531a7f5fd6b} | 0 ...egrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96} | 0 ...udf_div-0-31d31c7d5c544327dabfd874c88314db | 1 - ...df_div-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_div-1-31d31c7d5c544327dabfd874c88314db | 1 + ...udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 | 4 - .../udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b | 1 - ...udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 | 4 + ...df_div-3-5111340caad64e36370d9d4bc4db5f27} | 0 ..._divide-0-1af8b249439ee5b7d4978c31669bc208 | 1 - ..._divide-1-fa932664bae88683a222b71ac45fb840 | 4 - ..._divide-2-ce54773b1babe6dde982e4e91ebaeb50 | 1 - ..._double-0-aa32d73a99587cae2f1efd9a2094d617 | 2 +- ..._double-1-79380157cbd6624d760335f8291e6fb4 | 2 +- ...df_elt-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_elt-1-b46b060da76d1772db998c26a62a608f} | 0 ...udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a | 0 ...df_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4} | 0 ...df_elt-3-c2554fac72a2a51bb33faae16aec3507} | 0 ...df_elt-4-533ad9c703c320a6556c09dd5f9ac351} | 0 ..._equal-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._equal-1-36b6cdf7c5f68c91155569b1622f5876} | 0 ..._equal-2-2422b50b96502dde8b661acdfebd8892} | 0 ..._equal-3-e0faab0f5e736c24bcc5503aeac55053} | 0 ..._equal-4-39d8d6f197803de927f0af5409ec2f33} | 0 ..._equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca} | 0 ..._equal-6-878650cf21e9360a07d204c8ffb0cde7} | 0 ..._equal-7-1635ef051fecdfc7891d9f5a9a3a545e} | 0 ..._equal-8-276101b04b10b7cd6d59061a8cbf42d2} | 0 ...udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a | 2 +- ...udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 | 2 +- ...f_field-0-277b4a4dcb38cabb6df50147c77e0a33 | 1 - ..._field-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_field-1-277b4a4dcb38cabb6df50147c77e0a33 | 1 + ...f_field-1-379d8580693493f620a8f4084709324f | 2 - ..._field-10-7982ea72163dbc4cd45f53454edf66c8 | 2 - ...field-10-ca9db7e6bb687606bc273d1f6c191035} | 0 ..._field-11-7982ea72163dbc4cd45f53454edf66c8 | 2 + ...f_field-2-379d8580693493f620a8f4084709324f | 2 + ...f_field-2-d2c6583a79d77aabe388a52ec164c38b | 1 - ..._field-3-5960d42257b272f6ba043406229bbf26} | 0 ...f_field-3-fea09e934696af40bb604b40225bbc98 | 1 - ...f_field-4-212d8b2297bf6a3311d24d68de67b5c6 | 1 + ...f_field-4-b0815d34893d6cba8c07d0a0721c1d29 | 1 - ...f_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 | 0 ...f_field-5-949c2de113b214d83734c0c177f04b6b | 1 + ..._field-6-2d7c5cbe891c4a9dda34f9842f8e0828} | 0 ...f_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 | 0 ...f_field-7-3fd8b0c333acdf28c676315b03e2e10} | 0 ...f_field-7-6aa3518e9f55299754521e959e9376ef | 2 - ...f_field-8-66dc6c81db0ac9b2075783b0d8976083 | 0 ...f_field-8-6aa3518e9f55299754521e959e9376ef | 2 + ..._field-9-66dc6c81db0ac9b2075783b0d8976083} | 0 ...f_field-9-f053f2d16812aa60b6dd1cab61e90a95 | 0 ..._in_set-0-18d3e88b18c18a00598146a3307276f2 | 1 - ...in_set-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._in_set-1-18d3e88b18c18a00598146a3307276f2 | 1 + ..._in_set-1-5fb7a7a1725749dc3853eb80fba19246 | 7 - ...in_set-10-16355c6b7e169b3c0ef506c149c6853c | 1 - ...n_set-10-df21f44247d7275a292520c1605c4aab} | 0 ...in_set-11-5a8515684c458d3fffea539a3d170e3a | 1 - ...n_set-11-91f8c37820f31d0d1b16029a59a185ad} | 0 ..._in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 | 1 - ...n_set-12-692f41c998bbc6bec0f374eae4356739} | 0 ..._in_set-13-2c986a80620c9238e1f663fc591760a | 1 - ...in_set-13-45e5ae8f60de2c41f189db7922a04917 | 1 + ...in_set-14-189def133b1871ce8345a8123811a6b5 | 1 - ...n_set-14-8e410ecfad2d408ad7d2554ccd3a6621} | 0 ...in_set-15-671bff8f50feea55015a8412fc6e5ceb | 1 - ...n_set-15-c50e6ff95c05bfa854b33b03db858cd9} | 0 ...n_set-16-8e17f41ae6e8b1075af4790a8fd88e13} | 0 ...in_set-16-d5d22082588c5fc30ef502237c5797f4 | 1 - ...in_set-17-5b556a29e1685605bcc47bce60cf66c8 | 2 - ...n_set-17-fe61f992f2d971d006155bdec3143803} | 0 ...in_set-18-5b556a29e1685605bcc47bce60cf66c8 | 2 + ..._in_set-2-5fb7a7a1725749dc3853eb80fba19246 | 7 + ..._in_set-2-b3823bdc04a7f98951b55c3e30d2a772 | 0 ..._in_set-3-132b7bc7812db7683eb3bff607275d0e | 25 - ...in_set-3-b3823bdc04a7f98951b55c3e30d2a772} | 0 ..._in_set-4-132b7bc7812db7683eb3bff607275d0e | 25 + ..._in_set-4-a35471c87ba597a6d3c7c880704cac0b | 1 - ...in_set-5-6f25b5bba89e1fcae171f5d595acc4ee} | 0 ..._in_set-5-ddaa3551dffb1169b2fbb671f373b82f | 1 - ..._in_set-6-591e070365f19c65e453b98b88f5f823 | 1 - ..._in_set-6-7bf387b94afb755faca4ad73bb7c42ba | 1 + ..._in_set-7-72d05b5cf99388d539adec38c40978c3 | 1 - ...in_set-7-730d5e95ef748ad946eceefbcd633826} | 0 ..._in_set-8-780771cad9bec96a216aea8ab293c941 | 1 - ...in_set-8-ea11724531f191940e455d13878a0e69} | 0 ...in_set-9-81f9999ed1b063ce7f17d89bd0162777} | 0 ..._in_set-9-d59f5aabe1ea0963f9328065c699d175 | 1 - ...f_float-0-7987032f1c9dcad07001445f3ca1f7a7 | 2 +- ...f_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b | 2 +- ...f_floor-0-e35abe1d5534243e96d71bd0c28761d6 | 2 +- ...f_floor-1-497a1ddbcf738aead319fde4f90f5248 | 2 +- ...number-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._number-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._number-1-525f133cfff86d44afdeeda667c365a5 | 1 - ...number-1-e86d559aeb84a4cc017a103182c22bfb} | 0 ...number-10-3bddca6913ea7e281e223b0603010b77 | 1 + ..._number-2-525f133cfff86d44afdeeda667c365a5 | 1 + ..._number-2-591f302d5c1cd24e153a598fa0b352fd | 4 - ..._number-3-591f302d5c1cd24e153a598fa0b352fd | 4 + ..._number-3-c89564db1ab953e28b050b9740f2650c | 0 ..._number-4-295d41a2146a27320c2be90499343260 | 1 - ...number-4-7969ffc4e80f7214a8eead8e1084368a} | 0 ..._number-5-881f33f6727a30629bde6e4b178cf7d9 | 1 - ..._number-5-da5bf00d45d2bd758489f661caffd4dc | 1 + ..._number-6-6dfca21d142652fec9017ba828a226c8 | 1 + ..._number-6-a6720a128716e179e18933992ca899b3 | 1 - ..._number-7-31eda4b0f31406add3a61e2503590113 | 1 + ..._number-7-84a460780828b0b9a2235314cfc24766 | 1 - ..._number-8-b297476c6348209933132202030eb8ea | 1 + ..._number-8-e7eedc849c74ce7d33c559067dd9ca0e | 1 - ..._number-9-407a0a7c277bb4c5c94ce16533ce1646 | 1 - ..._number-9-a21fbe58ff475634c8ed9829c6b8c187 | 1 + ...nixtime-0-c3adaeede5c48d232473d78acf0eed7f | 2 +- ...nixtime-1-d1a511d2084c7c621b5f638908c8db65 | 2 +- ..._object-0-c08e7139c00878b98d396e65d958100f | 1 - ..._object-1-706bcfd51431ec7f2b80145837f94917 | 16 - ..._object-2-2a18d9570d9b676e240cda76df818c42 | 0 ..._object-3-f60851dc36f579e83d6848d7d3c589e6 | 0 ..._object-4-4f08101fd66fb25d7b322d47773e49f3 | 0 ...erthan-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...terthan-0-99d268829a124103cb3429c53fdc4de4 | 1 - ...terthan-1-8aab8e39726a986e10e1e572939fd63c | 1 - ...terthan-1-99d268829a124103cb3429c53fdc4de4 | 1 + ...terthan-2-79ba62f35a9432647e31c6722b44fe6f | 1 - ...terthan-2-8aab8e39726a986e10e1e572939fd63c | 1 + ...terthan-3-75fcadcdc6c050f1c7e70c71dc89c800 | 1 + ...requal-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...orequal-0-a7214027a91abf6501881e2af313347a | 1 - ...orequal-1-3669f2008e7f428f365efadbcb5ae451 | 1 - ...orequal-1-a7214027a91abf6501881e2af313347a | 1 + ...orequal-2-3669f2008e7f428f365efadbcb5ae451 | 1 + ...orequal-2-d2690cc7713e91201bb10ef291c95819 | 1 - ...orequal-3-631662997e0c8d24d80b5d64a17446d2 | 1 + ...f_hash-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_hash-1-b9e3a3986320d275982797140edfccf4} | 0 ...f_hash-2-a18646b51501d0b1beb967dc79afbd1a} | 0 ...df_hash-2-cc121f3c38a7a522abd824940fe04285 | 0 ...f_hash-3-2646a87ce26c383a9dafea9b56281ab7} | 0 ...f_hash-4-d1368c2e3cd113e46202156b44811987} | 0 ...df_hex-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 | 1 - ...udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 | 1 + ...udf_hex-1-d55348c0ccd133b7abb690f6949b520c | 8 - ...udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 | 1 - ...udf_hex-2-d55348c0ccd133b7abb690f6949b520c | 8 + ...udf_hex-3-3a1de5be8ce350612ee6a93303107470 | 1 + ...udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf | 1 - ...udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 | 1 + ...udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea | 1 - ...udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e | 1 + ...df_hour-0-ba1c46a403b807fe0a28b85e62d869ce | 1 - ...df_hour-1-3db41b9fe9966a45e663bc42cf182c04 | 7 - ...udf_hour-2-ace1054795b20abd5ae829814cfe15a | 0 ...df_hour-3-415b0842ab0818c82baf9fbf07180613 | 1 - ...udf_hour-4-73bfac513b993dedbe143306865a44a | 1 - ...udf_if-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_if-1-b7ffa85b5785cccef2af1b285348cc2c} | 0 ...udf_if-2-30cf7f51f92b5684e556deff3032d49a} | 0 .../udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 | 0 ...udf_if-3-59e90bb74481aaf35480076806daf365} | 0 .../udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca | 0 ...udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb} | 0 ...udf_if-5-841a8068d35a42179d3654e1a2848c43} | 0 ...udf_if-6-508f9140dd33931c7b9ad336dceb32cf} | 0 .../udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 | 1 - .../udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 | 3 - ...in_file-0-1775b929e50cae8b3e957d99c5378f59 | 1 - ...in_file-1-2f23153970a569a4643574dde8d78a58 | 0 ...f_index-0-a277ac394cae40cb55d1ef3aa5add260 | 1 - ..._index-0-e91e3e5a22029b9b979ccbbef97add66} | 0 ...f_index-1-1f5e109131b0c67ebea521fa4902a8f6 | 1 - ...f_index-1-a277ac394cae40cb55d1ef3aa5add260 | 1 + ...f_index-2-1f5e109131b0c67ebea521fa4902a8f6 | 1 + ...f_instr-0-2e76f819563dbaba4beb51e3a130b922 | 1 - ..._instr-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_instr-1-2e76f819563dbaba4beb51e3a130b922 | 1 + ...f_instr-1-32da357fc754badd6e3898dcc8989182 | 4 - ...f_instr-2-10147893f38fc08dad4fa7f2bc843fc2 | 0 ...f_instr-2-32da357fc754badd6e3898dcc8989182 | 4 + ...f_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 | 1 - ..._instr-3-c40fbd09410b11388ce7a6e9bea5846f} | 0 ...f_instr-4-7017a441a31abc235d9359440cefda49 | 1 + ...udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba | 2 +- ...udf_int-1-3f0405ff93adfe8b3402b118567867d7 | 2 +- ...notnull-0-44584503014c378bb916b38e1879bfb6 | 2 +- ...snotnull-1-6ca2ea7938c7dac331c603ad343c1a7 | 2 +- ..._isnull-0-ac8e7827d760108923509f9ea1691d53 | 2 +- ..._isnull-1-55d9d04204f30cde4aa2667db88db262 | 2 +- ...notnull-0-ac8e7827d760108923509f9ea1691d53 | 1 - ...notnull-1-55d9d04204f30cde4aa2667db88db262 | 1 - ...notnull-2-44584503014c378bb916b38e1879bfb6 | 1 - ...snotnull-3-6ca2ea7938c7dac331c603ad343c1a7 | 1 - ...notnull-4-3dd03048c0152f565b21b6d3b7b010f1 | 0 ...notnull-5-253ed8f6f8659120af927871f57d81a1 | 1 - ...notnull-6-9daf0ab0e3b8315018341d6f72bd3174 | 0 ...notnull-7-bb1030dea18d2a0c2c00a6e2de835d6b | 1 - ..._method-0-991b98a25032b21802bc2a1efde606c7 | 1 - ..._method-1-a3b94d9f2c2caf85a588b6686a64630a | 3 - ..._method-2-69e6b8725086a8fb8f55721705442112 | 0 ..._method-3-c526dfd4d9eac718ced9afb3cf9a62fd | 1 - ...f_lcase-0-649df2b742e6a03d0e0e364f5bee76ad | 2 +- ...f_lcase-1-d947c47e03bedbfd4954853cc134c66e | 2 +- ..._length-0-38364b60c3a2409f53c9aa2dae19903b | 1 - ...length-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._length-1-38364b60c3a2409f53c9aa2dae19903b | 1 + ..._length-1-f183e1f8ae516bb483132ed106289b67 | 4 - ...ength-10-de456a5765db4a06110d9483985aa4a6} | 0 ...length-10-f3a9bd30540345db0f69b6847014b333 | 1 - ...length-11-f3a9bd30540345db0f69b6847014b333 | 1 + ..._length-2-af46cb6887618240836eaf5be8afbba6 | 0 ..._length-2-f183e1f8ae516bb483132ed106289b67 | 4 + ...length-3-af46cb6887618240836eaf5be8afbba6} | 0 ..._length-3-dcd6404afce1103d5054527e6c216d6d | 0 ..._length-4-ba49ba4e6505c74bc33b5003f3930c43 | 0 ...length-4-dcd6404afce1103d5054527e6c216d6d} | 0 ..._length-5-adc1ec67836b26b60d8547c4996bfd8f | 25 - ...length-5-ba49ba4e6505c74bc33b5003f3930c43} | 0 ..._length-6-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ..._length-6-adc1ec67836b26b60d8547c4996bfd8f | 25 + ...length-7-460dffb0f1ab0ac0ebc4fd545809aa9a} | 0 ..._length-7-8f28e6c488df47d89dca670f04a7563f | 0 ..._length-8-5e0fe761b7520651c3446ce7f9179caf | 0 ...length-8-8f28e6c488df47d89dca670f04a7563f} | 0 ..._length-9-de456a5765db4a06110d9483985aa4a6 | 0 ...length-9-e41b220da98996f997b26ba7ef457a84} | 0 ...ssthan-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...essthan-0-a0d9e8b51e3d13685b3889db38f22427 | 1 - ...essthan-1-952c655a1092a410e5346f1205cb8142 | 1 - ...essthan-1-a0d9e8b51e3d13685b3889db38f22427 | 1 + ...essthan-2-92fa47f7af4a03ce1a965472eaad23a7 | 1 - ...essthan-2-952c655a1092a410e5346f1205cb8142 | 1 + ...essthan-3-677a1383983c94ba8008535b5a193153 | 1 + ...requal-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...orequal-0-b3b021456c85da76d1879191886a425b | 1 - ...orequal-1-869d983466744ad73b109211e1638869 | 1 - ...orequal-1-b3b021456c85da76d1879191886a425b | 1 + ...orequal-2-56775013e20ecf2287e07e83eccf2e0c | 1 - ...orequal-2-869d983466744ad73b109211e1638869 | 1 + ...orequal-3-947dd56091ae1ef399ab32ce58317667 | 1 + ...f_like-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_like-1-e0ba9a953e50554bdcbc55585cffde09} | 0 ...f_like-2-9781f89d352c506e972ad2a1d58ec03a} | 0 ...df_like-2-dbc46cb33f0dd356af03006d9492f8b7 | 0 ...f_like-3-dbc46cb33f0dd356af03006d9492f8b7} | 0 ...f_like-4-bef03784eab9d5e8404fd24960dea4fc} | 0 ...f_like-5-47bfd4d65090dab890b467ae06cf3bd5} | 0 .../udf_ln-0-779eed5722a0efaa85efe24c559072b4 | 2 +- .../udf_ln-1-60e3541b3c703d6413869d774df9b7e4 | 2 +- ...locate-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._locate-0-6e41693c9c6dceea4d7fab4c02884e4e | 1 - ..._locate-1-6e41693c9c6dceea4d7fab4c02884e4e | 1 + ..._locate-1-d9b5934457931447874d6bb7c13de478 | 4 - ..._locate-2-849e7058dbbd7d422a9f3eb08d85b15c | 0 ..._locate-2-d9b5934457931447874d6bb7c13de478 | 4 + ..._locate-3-2a260e4b8e909eb5e848bf31a07f2531 | 1 - ...locate-3-ce4a131f99dc9befa926027380b38dbb} | 0 ..._locate-4-104cbfb3b59ad563810ddd7304a58b1b | 1 + ...udf_log-0-ca773bc1afa66218f3c13dee676bd87a | 2 +- ...udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 | 2 +- ...f_log10-0-35028570b378a2c7ea25b6bf6a4fac1f | 2 +- ...f_log10-1-abf1173290ef905d24d422faf7801fe3 | 2 +- ...df_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 | 2 +- ...df_log2-1-a79f0dce2cfc000b11a3b5299f02db56 | 2 +- ...boolean-0-2e7b9484514a049bbf72a4a0af5ee127 | 10 - ...f_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 | 2 +- ...df_lower-1-550f0a6da388596a775d921b9da995c | 2 +- ...f_lower-3-61b2e3e72180c80d52cf9bed18125e08 | 2 +- ...f_lpad-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_lpad-0-995646acf1e23cea7825412915921bef | 1 - ...df_lpad-1-995646acf1e23cea7825412915921bef | 1 + ...df_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 | 6 - ...df_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 | 0 ...df_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 | 6 + ...df_lpad-3-5b04264ae9ada1304acd0410ce31f2ae | 1 - ...f_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7} | 0 ...df_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 | 1 + ...f_ltrim-0-398a623504c47bcd64fe8d200c41402f | 2 +- ...udf_ltrim-1-658d495908097792a0e33a77becac2 | 2 +- ...df_map-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_map-1-9feb9f29115f94b3bb4f6a36160bd17e} | 0 ...df_map-2-1f8cd98df9bf7b2528506551fef87dcf} | 0 ...udf_map-2-a3f90085abab46205e732b4c27b18340 | 0 ...df_map-3-be7b52baa973b8b59b7ca63fea19aa99} | 0 ...df_map-4-60cb9c30285f7a9f99377ccbd143eb06} | 0 ...ap_keys-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...ap_keys-1-9a5714f8790219e9a9708a2c45fc87aa | 1 - ...ap_keys-2-731b529a9a234473312b7d1db15be75f | 1 - ...ap_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e | 1 - ...ap_keys-4-10cb8da631c1c058dacbbe4834a5928a | 1 - ..._values-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._values-1-a1d9810d9793d4ca2d17f539ca72bd08 | 1 - ..._values-2-ed39a40cbe55bb33d2bc19f0941dae69 | 1 - ..._values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 | 1 - ..._values-4-a000d06dd3941756b4bb9ccc46f3620e | 1 - ...udf_max-0-ac7d002a46f773ab680ed8c1ac97821f | 1 - ...udf_max-1-14afa1f14687893233a662f0f32a40c9 | 1 - ...df_max-10-1e9209f5b2ba926df36b692a4dcd09f6 | 1 - ...df_max-11-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...df_max-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...df_max-13-1e9209f5b2ba926df36b692a4dcd09f6 | 1 - ...udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...udf_max-3-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 | 1 - ...udf_max-5-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...udf_max-6-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 | 1 - ...udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...udf_min-0-1a67398a47b4232c3786485b451d1ff8 | 1 - ...udf_min-1-69d749d0bca0ebe56e930009e30f4f19 | 1 - ...df_min-10-191613d4d46d1884d0694fcd8c5fb802 | 1 - ...df_min-11-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...df_min-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...df_min-13-191613d4d46d1884d0694fcd8c5fb802 | 1 - ...udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...udf_min-3-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...udf_min-4-191613d4d46d1884d0694fcd8c5fb802 | 1 - ...udf_min-5-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...udf_min-6-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...udf_min-7-191613d4d46d1884d0694fcd8c5fb802 | 1 - ...udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ..._minute-0-9a38997c1f41f4afe00faa0abc471aee | 2 +- ..._minute-1-16995573ac4f4a1b047ad6ee88699e48 | 2 +- ..._minute-3-270055c684846e87444b037226cf554c | 2 +- ..._modulo-0-4e06551d4aa9464492e0f53374a280d5 | 2 +- ..._modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 | 2 +- ...f_month-0-9a38997c1f41f4afe00faa0abc471aee | 2 +- ...f_month-1-16995573ac4f4a1b047ad6ee88699e48 | 2 +- ...negative-0-1b770ec6fb07bb771af2231a9723ec8 | 1 - ...gative-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...negative-1-1b770ec6fb07bb771af2231a9723ec8 | 1 + ...egative-1-5f64266721b1ed31cfe84ee2f2377bdf | 1 - ...ative-10-1cd28efecc0d26f463221195f5e39956} | 0 ...egative-2-5f64266721b1ed31cfe84ee2f2377bdf | 1 + ...egative-2-a6863d2c5fc8c3131fe70080a011392c | 1 - ...egative-3-a6863d2c5fc8c3131fe70080a011392c | 1 + ...egative-3-b90eec030fee9cbd177f9615b782d722 | 1 - ...egative-4-b90eec030fee9cbd177f9615b782d722 | 1 + ...egative-4-e27bf3f44ccb2e051877da8a4455f50c | 1 - ...gative-5-771e76b0acd8ddb128781da7819d0e47} | 0 ..._negative-5-93d7dd808d4af59bda601faf249a9e | 1 - ...egative-6-6758b00c5acc7aac320238accf299219 | 1 - ...gative-6-f62c4a097c592871d896a7dc47c42f61} | 0 ...negative-7-6d8783f0ed7a4b7058c95f90da3fb4b | 1 - ...gative-7-f838053f5ca5c8746dc299473dff0490} | 0 ...egative-8-634af0478ed9ed44b851cd7ef834a489 | 1 - ...gative-8-f4f23aa6f634913d194a69261af8f3f6} | 0 ...egative-9-80b4c1fe00f7997838bba64a2cb5f8aa | 1 - ...gative-9-f6a78fa3ea0f519d0e4abc5be7a960e5} | 0 ...udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc | 2 +- ...udf_not-1-efefc8302b02224d20f4bb0f159a6911 | 2 +- ...udf_not-2-7e63750d3027ced0e3452ad4eb0df117 | 2 +- ...udf_not-3-aa0c674f9ce0feba86448448a211bd2a | 2 +- ...otequal-0-27c0d57f7c7c48ceb7bc671f7160254e | 1 - ...tequal-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...otequal-1-27c0d57f7c7c48ceb7bc671f7160254e | 1 + ...otequal-1-a7f0d1064f8f398ef504008015fddf9a | 2 - ...otequal-2-7d815b1218c85e4cf69d6780cab17520 | 1 - ...otequal-2-a7f0d1064f8f398ef504008015fddf9a | 2 + ...otequal-3-7d815b1218c85e4cf69d6780cab17520 | 1 + ...otequal-3-b72baeb22fad61bb31ce2d2e69375f57 | 2 - ...otequal-4-b72baeb22fad61bb31ce2d2e69375f57 | 2 + ...otequal-4-eb04e5ee00967799c913e8a5b424a332 | 0 ...otequal-5-e361b9cf294c4aa25018b081a2c05e07 | 499 ---- ...tequal-5-eb04e5ee00967799c913e8a5b424a332} | 0 ...otequal-6-46a6514f2d7e6a097035ec1559df0096 | 0 ...otequal-6-e361b9cf294c4aa25018b081a2c05e07 | 499 ++++ ...tequal-7-46a6514f2d7e6a097035ec1559df0096} | 0 ...otequal-7-a71fea4e9514cda0da9542a7701613dd | 499 ---- ...otequal-8-a71fea4e9514cda0da9542a7701613dd | 499 ++++ ..._notop-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_notop-0-825431072651228a5a9de7f85a0498d6 | 1 - ...f_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 | 1 + ...df_nvl-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 | 1 - ...udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 | 4 - ...udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 | 1 + ...udf_nvl-2-175ed7006e8907b65e0e5357f00a0def | 0 ...udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 | 4 + ...df_nvl-3-47199a1c23cb1cc6827c601bb66513d3} | 0 ...udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 | 1 - ...udf_nvl-4-656661e80deb75729fef313d5e2bd330 | 1 + .../udf_or-0-c404aa929eb0dd87269121f8f99ada70 | 2 +- .../udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 | 2 +- ...rse_url-0-7571c0423df7bf158ea9ca98142b26b8 | 2 +- ...rse_url-1-67adfb10d4a35c4d031f26adde9f61ab | 2 +- ...rse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 | 2 +- ...centile-0-8f99f54ff944f252e47d0af1f4ed1553 | 1 - ...centile-1-c0825a744cd14917d2c904d014449a4a | 1 - ...centile-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...centile-3-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...df_pmod-0-ed67184beaf84c0542117c26651938e1 | 1 - ...df_pmod-1-90f75e01dcee85253a501d53b8562dae | 1 - ...f_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 | 1 - ...df_pmod-3-26d9546f030281a29a50a3e8e5858234 | 1 - ...df_pmod-4-7695df16d24a821224676e6bad3d66d1 | 1 - ...df_pmod-5-cf5311d51d44afb8d73f588e27d5e029 | 1 - ...udf_pmod-6-3c09a8da2f5645e732c22a45d055125 | 1 - ...df_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 | 1 - ...df_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd | 1 - ...df_pmod-9-e7280393102077442aa1d10eb69a6d57 | 1 - ...ositive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 | 2 +- ...ositive-1-1b98434a841d2248ed985c5f6ba2cc3c | 2 +- ...ositive-2-610d421e590f035c24e29694a68b0d23 | 2 +- ...ositive-3-400b238f4e6cdf7120be566b0ef079c5 | 2 +- ...udf_pow-0-c7f5178951dd45dc2a41c16729314d81 | 2 +- ...udf_pow-1-3c22c000c35144135aedbc7052f10803 | 2 +- ...df_power-0-57001d802c281743322d28bbc520cd4 | 2 +- ...f_power-1-ebd0398b2cb03f382a16382ddac13426 | 2 +- ..._printf-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._printf-1-19c61fce27310ab2590062d643f7b26e | 1 - ..._printf-2-25aa6950cae2bb781c336378f63ceaee | 4 - ..._printf-3-9c568a0473888396bd46507e8b330c36 | 0 ..._printf-4-91728e546b450bdcbb05ef30f13be475 | 1 - ..._printf-5-3141a0421605b091ee5a9e99d7d605fb | 1 - ..._printf-6-ec37b73012f3cbbbc0422744b0db8294 | 1 - ..._printf-7-5769f3a5b3300ca1d8b861229e976126 | 0 ...adians-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...radians-0-f899daf93b02ca681e0230a792c65e86 | 0 ...radians-1-58b73fc96927d447d1225f021eaa378} | 0 ...radians-1-eaaa62dd3935ff3152323dfafd136e93 | 1 - ...radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 5 + ...radians-2-bcaca433f704f71cf9d44c238a33c7b3 | 1 - ...radians-2-cb8462f25c18b7405c41a50e52476d04 | 1 + ...radians-3-65e16c7b13de48a5d36793d0c7d35e14 | 1 - ...radians-3-bd00297cb26f599913b14a635e768be3 | 1 + ...radians-4-65e16c7b13de48a5d36793d0c7d35e14 | 1 + ..._radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 4 - ..._radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 5 + ...radians-5-f899daf93b02ca681e0230a792c65e86 | 0 ...adians-6-70c9e7199b5898e2c3a4943ec58da113} | 0 ...radians-6-eaaa62dd3935ff3152323dfafd136e93 | 1 - ...radians-7-bcaca433f704f71cf9d44c238a33c7b3 | 1 - ...radians-7-cb8462f25c18b7405c41a50e52476d04 | 1 + ...radians-8-65e16c7b13de48a5d36793d0c7d35e14 | 1 - ...radians-8-bd00297cb26f599913b14a635e768be3 | 1 + ...radians-9-65e16c7b13de48a5d36793d0c7d35e14 | 1 + ..._radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 4 - ...df_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 | 2 +- ...df_rand-1-c6229b8f2ca3001663229cfb8ee4763e | 2 +- ...reflect-0-904138e2a1f831c308b7f0aacc859ae1 | 1 - ...reflect-1-21ec7caa253c7f95b7cf60191140e2ee | 4 - ...reflect-2-b868357466bab2f04685c2dc73604cf0 | 0 ...eflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 | 1 - ...eflect2-1-c5a05379f482215a5a484bed0299bf19 | 3 - ...eflect2-2-effc057c78c00b0af26a4ac0f5f116ca | 0 ...regexp-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...regexp-1-19917611f74aedc0922560f7f2595948} | 0 ...regexp-2-f7f0527cd47612d7f256edd5f8963800} | 0 ...regexp-3-59aff54bae544ee620141e4e629f167a} | 0 ...extract-0-e251e1a4b1e191814f26c54b14ab6cd9 | 2 +- ...extract-1-8add879ab5904bd805412ef8723276fb | 2 +- ...replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e | 2 +- ..._replace-1-e79b45aa220d3c4c3b4523ac9c897bc | 2 +- ...repeat-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._repeat-0-fdeae3e88f27ef148281d69ac8c4b23b | 1 - ..._repeat-1-836be47190989d8975a09a545ecbfe0b | 4 - ..._repeat-1-fdeae3e88f27ef148281d69ac8c4b23b | 1 + ..._repeat-2-836be47190989d8975a09a545ecbfe0b | 4 + ..._repeat-2-e1dbea7182ec1653e1123b5b67a6d20a | 0 ...repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec} | 0 ..._repeat-3-ba9dd02f59c74d63d60d60b6231a0365 | 1 - ..._repeat-4-64c5fce0c5ad4c26680a842aa740dc57 | 1 + ...f_rlike-0-6ec6ef55ac041208627454e16b501d38 | 2 +- ...f_rlike-1-829611a596e0c87431006f7247d25eca | 2 +- ...f_round-0-10b53ca1f15fd7879365926f86512d15 | 1 - ..._round-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_round-1-10b53ca1f15fd7879365926f86512d15 | 1 + ...f_round-1-2367bcc43510dedc80bdb6707e434da8 | 4 - ...f_round-2-2367bcc43510dedc80bdb6707e434da8 | 4 + ...f_round-2-9ffa2b573360cd879338de46d91ab374 | 1 - ...f_round-3-42a221909d3f7ed51bed01a65670461c | 1 - ...f_round-3-fa4d11da8e1eba258ed191ed5f1447de | 1 + ...f_round-4-b87ccaa1e0a87c558b56d59a8a074396 | 1 + ...f_round-4-dc80ec5189a4c6ce07688df3debe7de4 | 1 - ...f_round-5-441d0075081ae87579c959d714c4922d | 1 + ...f_round-5-a2414e7845ffafc61f75465508a1850a | 1 - ...f_round-6-4658ec3bc034b43e0477bf2474939449 | 1 + ...f_round-6-48439efa5c34e7589ab5003ed916f12b | 1 - ...df_round-7-74ff5a4862c80bd8fd84bede1a0320d | 1 + ...round_2-0-91afaf77ef4061fa20005a3c87dfef32 | 0 ...round_2-1-5e44354af73865d03e9088c0232f16ce | 0 ...round_2-2-83f91f60dcb2036b61b8b21f18281298 | 1 - ...round_2-3-c62cf7b74a91f605cf669e34a9315f93 | 1 - ...round_2-4-797fa8ed05cb953327d0f6306b30d6c3 | 1 - ...round_2-5-e41b862db8cc76c1fe004cf006fad50b | 1 - ...ound_3-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...round_3-0-8415af605db167315e4d9d3c69d89e6c | 1 - ...round_3-1-15a7f123f596e28e6f238063ba4e3d6d | 1 + ...round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 | 1 - ...round_3-2-5e7ada966f726ceb748f98c20eab4f10 | 1 - ...round_3-2-a03bf4e99027d4814a32c84d89d42cca | 1 + ...round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 | 1 + ...round_3-3-e94ab3326df006c7203ead86752f16a9 | 1 - ...round_3-4-8449fbdabbb4b1e6beab89be0af498f2 | 1 + ...round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 | 1 - ..._round_3-5-3844531c8cff115d6a33636db0a26ad | 1 + ...f_rpad-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_rpad-0-56de896c5fe8b40b22a9ed55ed79889c | 1 - ...df_rpad-1-48d17e5d5d6188300d048f987fab2ca0 | 6 - ...df_rpad-1-56de896c5fe8b40b22a9ed55ed79889c | 1 + ...udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 | 0 ...df_rpad-2-48d17e5d5d6188300d048f987fab2ca0 | 6 + ...f_rpad-3-66acb969c28a8e376782ccd0d442b450} | 0 ...df_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 | 1 - ...df_rpad-4-299dee5a72aad2a2738d7841a89bb71b | 1 + ...f_rtrim-0-7acca21c725995febdf2a9c1fdf0535a | 2 +- ...f_rtrim-1-66d61255134c09d37cbfedd757ae47fd | 2 +- ...second-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._second-0-e004a6f20da3fa6db868ec847b217ff2 | 1 - ..._second-1-3525f55f4f13253c42b3abaa53d77888 | 7 - ..._second-1-e004a6f20da3fa6db868ec847b217ff2 | 1 + ..._second-2-3525f55f4f13253c42b3abaa53d77888 | 7 + ..._second-2-d678372e3837a16be245d2e33482f17f | 0 ..._second-3-2496e4d3c64ca028184431c2930d82cf | 1 - ...second-3-d678372e3837a16be245d2e33482f17f} | 0 ..._second-4-2496e4d3c64ca028184431c2930d82cf | 1 + ...ntences-0-ec0dad44fa033691a731f6e4c6b5cf7f | 0 ...ntences-1-2dc07f4f0f0cb20d08c424e067ed8f69 | 0 ...tences-10-ce188a53f69129c14cbf378d2c3f6630 | 0 ...tences-11-3c8672506e23434187caf4e0064a8a80 | 31 - ...tences-12-d55c04a079ca97402509868f24921685 | 1 - ...ntences-2-1f218343f90e698fb9ed81c4f51d3d14 | 0 ...ntences-3-ce188a53f69129c14cbf378d2c3f6630 | 0 ...ntences-4-3c8672506e23434187caf4e0064a8a80 | 50 - ...ntences-5-60823f4912be26bee1a0b52a0a9588a9 | 0 ...ntences-6-27b7eeae683a87507f35e61fd4ce67de | 0 ...ntences-7-ec0dad44fa033691a731f6e4c6b5cf7f | 0 ...ntences-8-2dc07f4f0f0cb20d08c424e067ed8f69 | 0 ...ntences-9-68c61b4882802e416d5adaa2de440b59 | 0 ...df_sign-0-14f3c3034ac8039fc81681003bbf5e0e | 0 ...f_sign-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9} | 0 ...df_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 | 1 - ..._sign-10-9a5326b0bf612fed4ce0b04770bebc16} | 0 ...f_sign-10-fc7341f89c3cd3c43e436242d8aa61fa | 1 - ...f_sign-11-74237f5ecc497813cf9738b21647847a | 4 - ...f_sign-11-fc7341f89c3cd3c43e436242d8aa61fa | 1 + ...f_sign-12-74237f5ecc497813cf9738b21647847a | 4 + ...df_sign-2-85b743f0eed93904789cde4d1b5eafef | 1 - ...df_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 | 1 + ...df_sign-3-9eeb3985359429abba5d1dd702c66b0d | 1 - ...df_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 | 1 + ...f_sign-4-9a5326b0bf612fed4ce0b04770bebc16} | 0 ...df_sign-4-fc7341f89c3cd3c43e436242d8aa61fa | 1 - ...df_sign-5-74237f5ecc497813cf9738b21647847a | 4 - ...df_sign-5-fc7341f89c3cd3c43e436242d8aa61fa | 1 + ...df_sign-6-14f3c3034ac8039fc81681003bbf5e0e | 0 ...df_sign-6-74237f5ecc497813cf9738b21647847a | 4 + ...df_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 | 1 - ...f_sign-7-ed2aaa1a416c0cccc04de970424e1860} | 0 ...df_sign-8-85b743f0eed93904789cde4d1b5eafef | 1 - ...df_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 | 1 + ...df_sign-9-9eeb3985359429abba5d1dd702c66b0d | 1 - ...df_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 | 1 + ...udf_sin-0-40b50393869eb0bcde66e36fe41078ee | 1 - ...df_sin-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_sin-1-2f867f432fb322e21dce353d7eb50c63 | 4 - ...udf_sin-1-40b50393869eb0bcde66e36fe41078ee | 1 + ...udf_sin-2-2f867f432fb322e21dce353d7eb50c63 | 4 + ...udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab | 1 - ...df_sin-3-1d3a615e3aa252a317daa601811820b1} | 0 ...udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 | 1 - ...udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a | 1 + ...df_size-0-9c49a52514b1b940a7e8bb93c35eda62 | 1 - ...df_size-1-3608160636eaa7e053171bdcefc0b1a8 | 1 - ...df_size-2-96d41c9f054662827d1b6b63f5dd8db7 | 0 ...df_size-3-63df892215b6ce189d6c3e20cfc4bdbd | 1 - ...mallint-0-f28e857ef74c967303855c21dc60c042 | 2 +- ...mallint-1-37d8db74267ae370d6a076b3057c5ed6 | 2 +- ...t_array-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...t_array-1-976cd8b6b50a2748bbc768aa5e11cf82 | 1 - ...t_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 | 4 - ...t_array-3-55c4cdaf8438b06675d60848d68f35de | 0 ...t_array-4-3edb0151fae0622cb79cd04156cb4c44 | 1 - ...t_array-5-5d7dfaa9e4137938559eb536e28f8d0e | 1 - ...t_array-6-f754ac1296d16e168abb3d0ebcc35bd3 | 1 - ...t_array-7-a9c52a8684cefc095470f5a93f63d2a8 | 1 - ...t_array-8-d79f0084177230a7a2845c4791c22d25 | 0 ...t_array-9-45ef2679e195a269878527d5f264488a | 0 ..._space-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_space-0-91e879c9f01d90eab7bf12fcef256010 | 1 - ...f_space-1-91e879c9f01d90eab7bf12fcef256010 | 1 + ...f_space-1-e4eaf5e96807e122548cb43be9a26754 | 4 - ...f_space-2-a23a06eef37709e8587647a74bbfa7e8 | 0 ...f_space-2-e4eaf5e96807e122548cb43be9a26754 | 4 + ...f_space-3-59903e27d8188d6209e007ff643d5956 | 1 - ..._space-3-a1b9dad63547f7ba73a5230d650983b0} | 0 ...f_space-4-7adb35ad867ba3e980d937a0038ac1a5 | 1 - ...f_space-4-d9de5746edd753507c3f054e1bee7702 | 1 + ...f_space-5-ce5288dcc60f9412109930bd56752a65 | 1 + ...f_split-0-7accac7fc71ba74d61c01a69d3978338 | 1 - ...f_split-1-ebe303dfc3db504deffe5c355efd0fd1 | 4 - ...f_split-2-7bba11f8fc359f2d4b863cda11c643f9 | 0 ...f_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 | 1 - ...df_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 | 2 +- ...df_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 | 2 +- ...udf_std-0-e3613484de2b3fa707995720ec3f8a5b | 2 +- ...udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 | 2 +- ..._stddev-0-ad7627185d89a60b83ce19966eddbc92 | 2 +- ..._stddev-1-18e1d598820013453fad45852e1a303d | 2 +- ...dev_pop-0-96788538f1f20eb879a1add4bb5f9d12 | 2 +- ...dev_pop-1-6286ef94de26050617bf69c17a3b4a10 | 2 +- ...dev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 | 2 +- ...ev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 | 2 +- ...dev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 | 2 +- ...ev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 | 2 +- ..._string-0-17412ad1c1a827411caa7b5e891b6ac3 | 2 +- ..._string-1-53b00551846b7f8bb27874b3a466e68d | 2 +- ..._substr-0-20fb50d79b45264548b953e37d837fcd | 1 - ..._substr-1-2af34de7fd584c5f1ead85b3891b0920 | 10 - ..._substr-2-d0268ad73ab5d94af15d9e1703b424d1 | 1 - ..._substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 | 1 - ..._substr-4-f7933c0bb28e9a815555edfa3764524a | 1 - ..._substr-5-4f29c1273b8010ce2d05882fc44793d8 | 1 - ..._substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 | 1 - ..._substr-7-1742c72a1a8a6c6fd3eb2449d4747496 | 1 - ..._substr-8-ba6ca6bac87ca99aabd60b8e76537ade | 1 - ..._substr-9-a9aa82080133620d017160f6a644455d | 1 - ...bstring-0-8297700b238f417dea2bd60ba72a6ece | 2 +- ...bstring-1-e5df65419ecd9e837dadfcdd7f9074f8 | 2 +- ...ubtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 | 2 +- ...ubtract-1-b90eec030fee9cbd177f9615b782d722 | 2 +- ...udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 | 2 +- ...udf_sum-1-ddae1a511d3371122ab79918be5b495b | 2 +- ...udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 | 2 +- ...udf_sum-3-ddae1a511d3371122ab79918be5b495b | 2 +- ...df_tan-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_tan-0-c21aa640b4edabf6613dd705d029c878 | 1 - ...udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 - ...udf_tan-1-c21aa640b4edabf6613dd705d029c878 | 1 + ...udf_tan-2-27a29c68f846824990e9e443ac95da85 | 1 - ...udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 + ...udf_tan-3-77bedd6c76bdc33428d13a894f468a97 | 1 - ...df_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b} | 0 ...udf_tan-4-769fde617744ccfaa29cefec81b8704c | 1 + ...udf_tan-4-c21aa640b4edabf6613dd705d029c878 | 1 - ...udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 - ...udf_tan-5-c21aa640b4edabf6613dd705d029c878 | 1 + ...udf_tan-6-27a29c68f846824990e9e443ac95da85 | 1 - ...udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 + ...udf_tan-7-77bedd6c76bdc33428d13a894f468a97 | 1 - ...df_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b} | 0 ...udf_tan-8-769fde617744ccfaa29cefec81b8704c | 1 + ...tlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad | 10 - ...length2-0-3da4fe901124f2bbf3d02822652c4e55 | 10 - ...tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d | 2 +- ...tinyint-1-417de1aeb44510aa9746729f9ff3b426 | 2 +- ...o_byte-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 | 1 - ...o_byte-1-94eb069fb446b7758f7e06386486bec9} | 0 ...to_byte-2-233102b562824cf38010868478e91e1} | 0 ...o_byte-3-5dc0e4c21764683d98700860d2c8ab31} | 0 ...to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 | 1 - ...o_byte-4-dafb27507b4d30fd2231680f9ea80c82} | 0 ...o_byte-5-eb6600cd2260e8e75253e7844c0d7dc2} | 0 ...o_byte-6-489cd2d26b9efde2cdbff19254289371} | 0 ...o_byte-7-1eaba393f93af1763dd761172fb78d52} | 0 ...o_byte-8-8fe36cf8fba87514744a89fe50414f79} | 0 ...o_byte-9-322163c32973ccc3a5168463db7a8589} | 0 ...to_date-0-ab5c4edc1825010642bd24f4cfc26166 | 2 +- ...to_date-1-da3c817bc5f4458078c6199390ac915e | 4 +- ..._double-2-69bf8a5a4cb378bbd54c20cb8aa97abe | 1 - ..._double-3-ab23099412d24154ff369d8bd6bde89f | 1 - ..._double-4-293a639a2b61a11da6ca798c04624f68 | 1 - ..._double-5-42d1e80bb3324030c62a23c6d1b786a8 | 1 - ..._double-6-5bac1a7db00d788fd7d82e3a78e60be6 | 1 - ..._double-7-97080ab9cd416f8acd8803291e9dc9e5 | 1 - ..._double-8-df51146f6ef960c77cd1722191e4b982 | 1 - ...o_float-2-39a67183b6d2a4da005baed849c5e971 | 1 - ...o_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce | 1 - ...o_float-4-f3e0ab8ed691a386e6be4ce6993be507 | 1 - ...o_float-5-75f364708c01b5e31f988f19e52b2201 | 1 - ...o_float-6-1d9b690354f7b04df660a9e3c448a002 | 1 - ...o_float-7-cdfefa5173854b647a76383300f8b9d1 | 1 - ...o_float-8-5b0a785185bcaa98b581c5b3dbb3e12c | 1 - ...o_long-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...to_long-1-61e6679e5a37889bc596590bde0228f0 | 1 - ...o_long-1-8aeb3ba62d4a0cecfff363741e8042f6} | 0 ...o_long-2-7e8fa1ae8d00a121ec14941a48d24947} | 0 ...o_long-3-6e5936fba8e7486beb9ab998548bbe9b} | 0 ...o_long-4-8c284b082a256abf0426d4f6f1971703} | 0 ...o_long-5-6bb29b93f6b0f1427ba93efb4e78810a} | 0 ...o_long-6-290b5a4ce01563482e81b3b532ebf9db} | 0 ...o_long-7-da20f84586dac3e50ee9d5b9078f44db} | 0 ...o_long-8-90f068b4b6275bdd1c4c431fb7fa90e2} | 0 ...o_long-9-cc8b79539085fe0e00f672b562c51cd0} | 0 ..._short-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...o_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a | 1 - ...o_short-1-32c4a61f9166c2a9d7038553d589bd9b | 1 - ..._short-1-94f15fe043839493107058a06a210cf7} | 0 ..._short-2-981e4cb6654fde7eb4634c7ad72f8570} | 0 ..._short-3-b259ee30ecf279bb4ad12d1515ca2767} | 0 ...o_short-4-40ffb132d5641645e2b8043dc056fb0} | 0 ..._short-5-5d6c46b0154d1073c035a79dbf612479} | 0 ..._short-6-6561b41835a21f973cbbc2dd80eef87f} | 0 ..._short-7-9f83813005b639a23901ca6ff87ff473} | 0 ..._short-8-885656e165feb3a674cf636dbf08716c} | 0 ..._short-9-750382fa1a1b3ed5dca0d549d3a68996} | 0 ...o_string-0-a032eb8f819689a374852c20336d5cc | 1 - ..._string-1-b461f0e6e98991aede40c7c68561dc44 | 1 - ..._string-2-bc6b37e1d5a8d28df8a957c88104c9a5 | 1 - ..._string-3-ddbb829eedc6aa1e27169303a7957d15 | 1 - ..._string-4-c20301e9bbf10143bb9bf67cd7367c21 | 1 - ...to_string-5-1040b37847d20ef29d545934316303 | 1 - ..._string-6-4181d264a7af3c6488da2f1db20dc384 | 1 - ..._string-7-567bc77f96e7dc8c89bae912e9a3af15 | 1 - ..._string-8-a70b03d79ebd989c8141f9d70dbca8ea | 1 - ..._string-9-51677fbf5d2fc5478853cec1df039e3b | 1 - ...nslate-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...anslate-0-7fe940a2b26fa19a3cfee39e56fb1241 | 1 - ...anslate-1-7fe940a2b26fa19a3cfee39e56fb1241 | 1 + ...anslate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 | 13 - ...nslate-10-2f9daada8878081cb8954880ad5a65c4 | 1 - ...nslate-10-ca7c17e78c6a3d4e19dbd66622a87eae | 1 + ...nslate-11-40c4e7adff4dde739d7797d212892c5a | 1 + ...nslate-11-76b7a339d5c62808b9f4f78816d4c55b | 1 - ...nslate-12-a5b3e5fadeec1e03371160879f060b05 | 1 - ...nslate-12-d81fd3267ec96cff31079142bf5d49bf | 1 + ...nslate-13-26085a3eba1a1b34684ec4e6c1723527 | 1 + ...anslate-2-42aba80bf1913dd7c64545831f476c58 | 0 ...anslate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 | 13 + ...anslate-3-20904c8be8fed5cbd2d66ead6248a60a | 0 ...nslate-3-42aba80bf1913dd7c64545831f476c58} | 0 ...nslate-4-20904c8be8fed5cbd2d66ead6248a60a} | 0 ...anslate-4-5d4abaf86254bacaa545c769bd7e50ba | 0 ...nslate-5-5d4abaf86254bacaa545c769bd7e50ba} | 0 ...anslate-5-f2637240d227f9732d3db76f2e9d3a59 | 0 ...anslate-6-55122cc5ea4f49e737fee58945f3f21b | 1 - ...nslate-6-f2637240d227f9732d3db76f2e9d3a59} | 0 ...anslate-7-856c9d0157c34ab85cc6c83d560bfd47 | 1 - ...anslate-7-f8de3ab54db5d6a44fddb542b3d99704 | 1 + ...anslate-8-1747ed8fbb4ef889df3db937ee51e2b0 | 1 + ...anslate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 | 1 - ...anslate-9-ee69663d7662760973b72785595be2b1 | 1 - ...anslate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 | 1 + ...df_trim-0-18aa2b7ff8d263124ea47c9e27fc672f | 2 +- ...df_trim-1-e23715e112959e6840b6feed2ecf38a7 | 2 +- ...f_ucase-0-8f8c18102eb02df524106be5ea49f23d | 2 +- ...f_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f | 2 +- ...f_unhex-0-11eb3cc5216d5446f4165007203acc47 | 1 - ...f_unhex-1-a660886085b8651852b9b77934848ae4 | 14 - ...df_unhex-2-78ba44cd7dae6619772c7620cb39b68 | 1 - ...f_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 | 1 - ...f_union-0-e3498ef95bc9d8c8ce55e75962b4a92c | 1 - ...f_union-1-f6f01250718087029165e23badc02cd6 | 4 - ...f_union-2-6af20858e3209d7cf37f736805ec5182 | 0 ...f_union-3-705d165fec6761744dd19b142c566d61 | 2 - ...estamp-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...estamp-1-d555c8cd733572bfa8cd3362da9480cb} | 0 ...mestamp-2-28c40e51e55bed62693e626efda5d9c5 | 0 ...estamp-2-8a9dbadae706047715cf5f903ff4a724} | 0 ...estamp-3-28c40e51e55bed62693e626efda5d9c5} | 0 ...mestamp-3-732b21d386f2002b87eaf02d0b9951ed | 0 ...estamp-4-6059ff48788d0fb8317fd331172ecea9} | 0 ...estamp-5-b2e42ebb75cecf09961d36587797f6d0} | 0 ...estamp-6-31243f5cb64356425b9f95ba011ac9d6} | 0 ...estamp-7-9b0f20bde1aaf9102b67a5498b167f31} | 0 ...estamp-8-47f433ff6ccce4c666440cc1a228a96d} | 0 ...f_upper-0-47dc226b0435f668df20fe0e84293ead | 2 +- ...f_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d | 2 +- ...var_pop-0-3187e740690ccc1988a19fea4202a6de | 2 +- ...var_pop-1-fd25e5226312bf54d918858511814766 | 2 +- ...ar_samp-0-b918928871d1b7f944315558c230c229 | 2 +- ...ar_samp-1-59032ed5856fd4aa17c3e8e6721eec2b | 2 +- ...ar_samp-2-b918928871d1b7f944315558c230c229 | 2 +- ...ar_samp-3-59032ed5856fd4aa17c3e8e6721eec2b | 2 +- ...ariance-0-fd23be1baa8b5ffa0d4519560d3fca87 | 2 +- ...ariance-1-c1856abae908b05bfd6183189b4fd06a | 2 +- ...ariance-2-3187e740690ccc1988a19fea4202a6de | 2 +- ...ariance-3-fd25e5226312bf54d918858511814766 | 2 +- ...ariance-4-fd23be1baa8b5ffa0d4519560d3fca87 | 2 +- ...ariance-5-c1856abae908b05bfd6183189b4fd06a | 2 +- ...ariance-6-3187e740690ccc1988a19fea4202a6de | 2 +- ...ariance-7-fd25e5226312bf54d918858511814766 | 2 +- ...ofyear-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...kofyear-0-d6b4490b549a358be375511e39627dc2 | 1 - ...kofyear-1-b7bbdfabe6054a66701250fd70065ddd | 6 - ...kofyear-1-d6b4490b549a358be375511e39627dc2 | 1 + ...kofyear-2-57ec578b3e226b6971e0fc0694b513d6 | 1 - ...kofyear-2-b7bbdfabe6054a66701250fd70065ddd | 6 + ...kofyear-3-d5dd3abb6c8c7046a85dd05f51126285 | 1 + ...f_when-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_when-0-88b97c6722176393e9b3d089559d2d11 | 1 - ...df_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 | 1 - ...df_when-1-88b97c6722176393e9b3d089559d2d11 | 1 + ...df_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 | 1 + ...df_when-2-ff1118e7d06a4725e1e98a6d70a59295 | 0 ...f_when-3-734890c41528b9d918db66b0582228a4} | 0 ...df_when-3-e63043e8d6ecf300c1fcf8654176896f | 1 - ...df_when-4-c57d6eb11efc29ce3a9c450488f3d750 | 1 + ...df_when-5-6ed21e998c0fc32c39f6375136f55de6 | 1 + ..._xpath-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 | 1 - ...f_xpath-1-5a6f10392053a4eabe62f5cabb073a71 | 8 - ...df_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 | 1 + ...f_xpath-2-5a6f10392053a4eabe62f5cabb073a71 | 8 + ...f_xpath-2-6b0a9d3874868d88d54ae133c978753d | 1 - ...f_xpath-3-5700d81a9c2a22dcca287bf8439277ea | 1 - ...f_xpath-3-f0b9adf99c68290c86d0f40f45111e18 | 1 + ...f_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a | 1 - ...f_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e | 1 + ...f_xpath-5-7395e1cd3b543316a753978f556975e0 | 1 - ...f_xpath-5-b66a64f91dd812fda2859863855988b5 | 1 + ...f_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f | 1 - ...f_xpath-6-d4f95ebc0057639138900722c74ee17a | 1 + ...f_xpath-7-6b8fceac3bd654662f067da60670e1d9 | 1 + ...oolean-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 | 1 - ...boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 | 1 + ...boolean-1-9e935539c9a3f3a118145096e7f978c4 | 6 - ...boolean-2-66148c16fde3008348a24cec5d9ba02b | 1 - ...boolean-2-9e935539c9a3f3a118145096e7f978c4 | 6 + ...oolean-3-9b97a08303a7a89e7575687f6d7ba435} | 0 ...boolean-3-c6cf7ede46f0412fe7a37ac52061b060 | 1 - ...oolean-4-284ce7d6fc6850ca853111145784286b} | 0 ...boolean-4-45d2e8baee72a0d741eb0a976af3a965 | 1 - ...oolean-5-95a0dfa897ba9050ad751a78aeb72f3d} | 0 ...boolean-5-dbc57416174a6ba26f29dfc13f91f302 | 1 - ...boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f | 1 - ...oolean-6-796c717c9d52d3efa374d12fe65259e6} | 0 ...boolean-7-824c07ed4ef40cd509fea55575e43303 | 1 - ...boolean-7-cc3ae9a7505e04a2e9b950442a81a559 | 1 + ...boolean-8-62a1af222d4e12c59cef71b979c6e58} | 0 ..._double-0-39199612969071d58b24034a2d17ca67 | 1 - ...double-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._double-1-2d5ceab0a14d6e837ed153e1a2235bb2 | 5 - ..._double-1-39199612969071d58b24034a2d17ca67 | 1 + ...double-10-ad0f9117f6f52590d38e619e998a6648 | 1 + ...double-10-fe9ae5013ee4f11b357128ee5ffb56c0 | 1 - ...double-11-a1960676c1b2dc76aa178ea288d801d2 | 1 - ...double-11-bf1f96ce71879c17d91ac9df44c36d29 | 1 + ...double-12-9621353ce5c583ca34216d357c5bb0eb | 1 + ..._double-2-2d5ceab0a14d6e837ed153e1a2235bb2 | 5 + ..._double-2-a4d22cea9dffaf26b485b3eb08963789 | 1 - ..._double-3-a4d22cea9dffaf26b485b3eb08963789 | 1 + ..._double-3-ea050c5b9e74d4b87b037236ef9e4fc2 | 5 - ..._double-4-93f47057c68385cff3b6f5c42307590c | 1 - ..._double-4-ea050c5b9e74d4b87b037236ef9e4fc2 | 5 + ..._double-5-1e0514d71f99da09e01a414a4e01d046 | 1 + ..._double-5-c811a2353f5baf585da8654acd13b0e5 | 1 - ..._double-6-10fcb281fd8db12dd1ac41380b8030c6 | 1 - ..._double-6-6a3985167fedd579f7bd5745133a3524 | 1 + ..._double-7-4532934141d2adb1f4ff56e65e8cf14c | 1 - ..._double-7-d97e93fb4b69522300f505e04b6674c8 | 1 + ..._double-8-547e750f5e401511db56283e66d1231d | 1 - ..._double-8-fce6cafa07b75c9843c1c1964e84fc10 | 1 + ..._double-9-b45b8b60031ac43ed2ffcd883ba2f19e | 1 - ..._double-9-f27361521c35bf4f6581dba4c185d550 | 1 + ..._float-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...h_float-0-7483dafee0dc7334eecabba31977e791 | 1 - ...h_float-1-7483dafee0dc7334eecabba31977e791 | 1 + ...h_float-1-f6ddff2daba006787aeb861ca6f9d37a | 4 - ..._float-10-89ae28cf8e6b6f422d307a0085774cab | 1 + ...h_float-2-70b7180f7bcd1d09727ad73084788d16 | 1 - ...h_float-2-f6ddff2daba006787aeb861ca6f9d37a | 4 + ...h_float-3-79b7e57efe31a0373c39f0ba79b5bd54 | 1 - ...h_float-3-b743a9cb9f8688220e0a6346856f18ce | 1 + ...h_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 | 1 + ...h_float-4-6e1f61ebe10c5fde60148e3a31706352 | 1 - ...h_float-5-410760f9560157587fbba7a677e12b9f | 1 - ...h_float-5-f5e1100f6e8de31081042413b4039fb2 | 1 + ...h_float-6-5e8457209d15467af7f14c09dfadb894 | 1 + ...h_float-6-d83a5002f52e3acb7dbb725bb434eebf | 1 - ...h_float-7-580ffe4dabef758c4fcb52050b315615 | 1 + ...h_float-7-b57c58f9343a82846f54ef60b216dfaa | 1 - ...h_float-8-3a62c730d4e5d9be6fb01307577d6f48 | 1 - ...h_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 | 1 + ...h_float-9-16793f703b552fcb24f7aea9bb8a2401 | 1 - ...h_float-9-bf66b884da717d46f0b12c3b8cf8313a | 1 + ...th_int-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...ath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 | 1 - ...ath_int-1-26ddf7e46a66065909e9e547f0459910 | 4 - ...ath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 | 1 + ...th_int-10-ea25feb474c40c6985152093d4dbb13a | 1 + ...ath_int-2-26ddf7e46a66065909e9e547f0459910 | 4 + ...ath_int-2-f10e246ebfd5f82545a3df64f51f58ba | 1 - ...th_int-3-a04ed6832ab542d6ee5903039511a826} | 0 ...ath_int-3-eaa4a790c726cfa76a247d08a6ee7aba | 1 - ...ath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 | 1 - ...th_int-4-bf5a4dbb7a98abc91111a3798b56809f} | 0 ...ath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 | 1 - ...ath_int-5-f49db0ecb889722ec68d1753c084b0e1 | 1 + ...ath_int-6-9bfa4fdc7d551742ff23efad8507ba0a | 1 - ...th_int-6-ac509f06f01c02924adef220404fc515} | 0 ...th_int-7-87ff12d650afb8f511d906778450fae7} | 0 ...ath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 | 1 - ...ath_int-8-a175811eca252aa77c16a44fbb0ee7b2 | 1 + ...ath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 | 1 - ...ath_int-9-7da88f589199f5ca873780fb22614573 | 1 + ...ath_int-9-e6c2548098879ee747a933c755c4c869 | 1 - ...h_long-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...th_long-0-d274b272251e95ec2e8139bfa471bd0b | 1 - ...th_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 | 4 - ...th_long-1-d274b272251e95ec2e8139bfa471bd0b | 1 + ...h_long-10-caeac94758a40493a5227fcdb8ec2f87 | 1 + ...th_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 | 4 + ...th_long-2-d697d943b1f7e7762d804064d11b905b | 1 - ...h_long-3-9df8d27e31d96e0b35b9b40910d4bd98} | 0 ...th_long-3-c0b20c651ae5a352322067f97bd6ae5d | 1 - ...h_long-4-3211913c56521887d30e3d1a50762b3f} | 0 ...ath_long-4-821e1cdea837bee7a8c2745bc3b85b9 | 1 - ...th_long-5-22a71b88c5bcb3db6e299a88ab791d4d | 1 - ...th_long-5-d580a8f01a546baddd939b95722e6354 | 1 + ...h_long-6-b695348ed3faec63be2c07d0d4afaaf3} | 0 ...th_long-6-f2460325cf46c830631d8bc32565c787 | 1 - ...th_long-7-b3f1d4b505151180b82fddb18cf795d0 | 1 - ...h_long-7-ed5af6d7451107a753b2c7ff130ac73b} | 0 ...th_long-8-2f952b56682969bb203fa3d9102f7015 | 1 + ...th_long-8-c21bebec7d1a4aec99fba6b0a9a03083 | 1 - ...th_long-9-7bafedc7b884df49a9f6752360969bf1 | 1 - ...th_long-9-947b6e08ba9c7defd75d00412f9bc4fd | 1 + ..._short-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...h_short-0-7d5231aed9cbbf68cd740791f9e5be17 | 1 - ...th_short-1-5d45932563b78e0b435b8cfebfe3cc2 | 4 - ...h_short-1-7d5231aed9cbbf68cd740791f9e5be17 | 1 + ..._short-10-b537709676634250e13914e76cd9a530 | 1 + ...h_short-2-10e9d4899d2fd352b58010c778c1f7a8 | 1 - ...th_short-2-5d45932563b78e0b435b8cfebfe3cc2 | 4 + ...h_short-3-5ac84f46610107b1846f4a9b26a69576 | 1 - ..._short-3-c09b4ae6886fa58dcdd728bef45e7efa} | 0 ..._short-4-84f5f6bb2fdc0987d281d52a53a4b24e} | 0 ...h_short-4-8a300079521fefbe0d2f943851c1c53c | 1 - ...h_short-5-51f5de44cf1d5289fa5892ffe16e473e | 1 - ..._short-5-c09fd0565ed041c773fee9bd0436e861} | 0 ..._short-6-16ced3de15d4ec87a4e7001376551758} | 0 ...h_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 | 1 - ..._short-7-8ffdf20c15f3ed81bb5a92c61d200ae2} | 0 ...h_short-7-e24ee123f331429c22de0a06054d0d5d | 1 - ...h_short-8-102ad2dea8d94528b402d980a45d53d4 | 1 + ...h_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 | 1 - ...h_short-9-22088ba0fb00eaa28e3460ca018b343e | 1 - ...h_short-9-d571e18b7d8ad44fef2e0b2424f34a0d | 1 + ...string-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...string-1-e315d11e9feb29177b5cb1e221c7cfa4} | 0 ...tring-10-d87fb71039c9d2419d750a0721c5696f} | 0 ...string-2-110b583cde6cd23c486d8223c444cbe9} | 0 ...string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4} | 0 ...string-4-302630fe7dac2cc61fe7d36ead0f41ab} | 0 ...string-5-19357ba9cb87d3a5717543d2afdc96e2} | 0 ...string-6-4837e470f745487fae4f498b3a2946bc} | 0 ...string-7-8e42951d002e3c4034b4a51928442706} | 0 ...string-8-fdfa4e17d70608dcc634c9e1e8a8f288} | 0 ...string-9-84109613320bd05abccd1058044d62c3} | 0 ...f_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a | 1 - ...f_stack-1-879ca1a8453ced55a8617b390670a4e1 | 0 ...f_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 | 0 ...otation-1-3de206f543c9e1525c54547f076b99c3 | 10 +- ...otation-4-3de206f543c9e1525c54547f076b99c3 | 10 +- ...otation-7-3de206f543c9e1525c54547f076b99c3 | 10 +- ...union10-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...union10-4-7f83822f19aa9b973198fe4c42c66856 | 2 +- ...union11-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...union11-2-cd756f39e22e121bdbd51400662aa47f | 2 +- ...union12-0-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...union12-1-9d2793d1cfd2645ac7f373a0a127e599 | 0 ...union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 | 0 ...union13-1-534d0853c5fc094404f65ca4631c1c20 | 2 +- ...union14-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...union14-2-8e01b2f4a18ad41a622e0aadbe680398 | 2 +- ...union15-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...union15-2-a89acfb4bbc044c483b94e28152a41e0 | 2 +- ...union16-1-9f76074598f9b55d8afbb5659737a382 | 2 +- ...union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...union17-5-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...union18-4-1799ebb147238db6032fd6fe2fd36878 | 2 +- ...union18-5-b12dcddfa4f02a14318f6564947c98a0 | 2 +- ...union19-4-1799ebb147238db6032fd6fe2fd36878 | 2 +- ...union19-5-b12dcddfa4f02a14318f6564947c98a0 | 2 +- .../union2-1-90d739774cb96e7d0d96513c1c9968b4 | 2 +- ...union20-1-968e353589f1fddb914242beb25be94c | 2 +- ...union21-0-ecfd22e2a24ed9f113229c80a2aaee9c | 0 ...union21-1-fb1497f4c21bf7d28162f27d50320d13 | 536 ---- ...union23-1-7830963417e3535034962e2597970ddd | 2 +- ...union27-3-ab84df3813ff23be99f148449610e530 | 2 +- ...union28-3-b1d75ba0d33a452619e41f70e69616e9 | 2 +- ...union29-3-b1d75ba0d33a452619e41f70e69616e9 | 2 +- ...union30-3-b1d75ba0d33a452619e41f70e69616e9 | 2 +- ...nion31-14-c36a1d8de2713f722ec42bc4686d6125 | 2 +- ...nion31-15-5df6435aed6e0a6a6853480a027b911e | 2 +- ...nion31-24-df38c8164af7cc164c728b8178da72c5 | 2 +- ...nion31-25-1485e295a99908e1862eae397b814045 | 2 +- ...union31-8-ba92b89786ffaecd74a740705e0fa0cb | 2 +- ...union31-9-56dfdb30edd8a687f9aa9cad29b42760 | 2 +- ...union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e | 0 ...union32-1-e6c80e7d6171ae5fc428506e57dc8753 | 0 ...nion32-10-44a9b373ee7d43a4ef2bc4b8a708601b | 0 ...nion32-11-40d51bc9958b27c36ef647f0598fdee5 | 32 - ...union32-2-8e39fef33c859ef83912d0fcda319218 | 0 ...union32-3-d31e252450077ac54f4cb18a9ad95a84 | 20 - ...union32-4-79787e084ca15d479cee3a7e1ed2281e | 0 ...union32-5-51c997d0a1103b60764bbb8316a38746 | 32 - ...union32-6-96fa13d8790bbfa1d6109b7cbf890d1b | 0 ...union32-7-f936440d63f4e1027dda4de605660633 | 32 - ...union32-8-f42d83f502a7b8d3a36331e0f5621cfb | 0 ...union32-9-74fb695786df4c024288ae23ac8c00db | 32 - ...nion34-10-da2b79118c21ac45ce85001fa61b0043 | 2 +- ...union34-4-101829a66cab2efd31dcb0d86e302956 | 0 ...nion34-4-70479e10c016e5ac448394dbadb32794} | 0 ...union34-5-24ca942f094b14b92086305cc125e833 | 2 +- ...union34-7-da2b79118c21ac45ce85001fa61b0043 | 2 +- ...union34-8-b1e2ade89ae898650f0be4f796d8947b | 2 +- .../union4-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- .../union4-4-7f83822f19aa9b973198fe4c42c66856 | 2 +- .../union5-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- .../union5-2-2c19c8d564b010eeb42deee63d66a292 | 2 +- .../union6-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- .../union6-4-a03959cc5aaa8f6521a73e6dae04cd15 | 2 +- .../union7-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- .../union7-2-55d6e503a281acf3289a7874c0fba3f5 | 2 +- .../union8-1-1b422e4c1c8b97775518f760b995c771 | 2 +- .../union9-1-a77ee9f723b3b17a3a02164c5d0000c1 | 2 +- ...ion_date-4-d812f7feef3b6857aeca9007f0af44c | 0 ...n_date-4-d85fe746334b430941c5db3665e744d4} | 0 ...n_date-5-82eebfded24cef08e0a881d1bcca02b1} | 0 ...on_date-5-b54839e0200bec94aa751fec8c5dbd3d | 0 ...on_null-0-27e98c4939abf1ad4445b4e715b0262a | 10 - ...remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_1-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_1-12-43d53504df013e6b35f81811138a167a | 1 - ...emove_1-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_1-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_1-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_1-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...remove_1-6-a6c043a89a9c3456af8ee065cb17239 | 0 ...emove_1-7-48f70528347f5201f387d28dae37a14a | 0 ...emove_1-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_1-9-19865a08066d80cb069ae6312c465ee6 | 0 ...emove_10-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_10-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_10-10-7eaf13bc61bd4b43f4da67c347768598 | 0 ...ove_10-11-b62595b91d2d9e03a010b49ab81725d5 | 0 ...move_10-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_10-13-43d53504df013e6b35f81811138a167a | 1 - ...move_10-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_10-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_10-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_10-5-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_10-6-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_10-7-a7ba0436265932086d2a2e228356971 | 0 ...move_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...move_10-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_11-0-3c29684bfd2df7439ee0551eb42cfa0 | 2 +- ...move_11-1-16a6a293f1d2ce481b1d2482b1d5787c | 2 +- ...move_11-12-ea111d286c70e4a0c6a68a7420dc7b7 | 17 +- ...ove_11-13-43d53504df013e6b35f81811138a167a | 2 +- ...ove_11-14-e409e7032445097ace016b1876d95b3e | 2 +- ...move_11-2-cafed8ca348b243372b9114910be1557 | 2 +- ...move_11-3-b12e5c70d6d29757471b900b6160fa8a | 2 +- ...move_11-4-593999fae618b6b38322bc9ae4e0c027 | 2 +- ...move_11-5-6f53d5613262d393d82d159ec5dc16dc | 2 +- ...move_11-6-a2a411ad6620aa1ab24550ade336e785 | 2 +- ...move_11-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_11-9-94da21f150ed2c56046b80e46da8884d} | 0 ...emove_12-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_12-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_12-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_12-11-a667f24e26435cd2a29fef0ee45bab3c | 0 ...ove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 | 0 ...move_12-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_12-14-43d53504df013e6b35f81811138a167a | 1 - ...move_12-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_12-3-24ca942f094b14b92086305cc125e833 | 1 - ...move_12-4-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_12-5-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_12-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_12-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_12-8-9dd030d38eece4630dec5951fc8a0622 | 0 ...move_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_13-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_13-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_13-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 | 0 ...ove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af | 0 ...move_13-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_13-14-43d53504df013e6b35f81811138a167a | 1 - ...move_13-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_13-3-24ca942f094b14b92086305cc125e833 | 1 - ...move_13-4-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_13-5-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_13-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_13-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_13-8-234ecbc3f8c0e4686d3586f81cf191eb | 0 ...move_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_14-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_14-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_14-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_14-11-a667f24e26435cd2a29fef0ee45bab3c | 0 ...ove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 | 0 ...move_14-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_14-14-43d53504df013e6b35f81811138a167a | 1 - ...move_14-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_14-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_14-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_14-5-24ca942f094b14b92086305cc125e833 | 1 - ...move_14-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_14-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_14-8-ed33d620523b2634285698a83f433b6d | 0 ...move_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_15-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_15-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_15-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_15-11-93b7341b523213ab6e58169459bc6818 | 0 ...ove_15-12-616cc477ed00e691dbc2b310d1c6dd12 | 0 ...move_15-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 - ...ove_15-14-37f9f7bc2d7456046a9f967347337e47 | 2 - ...ove_15-15-43d53504df013e6b35f81811138a167a | 1 - ...move_15-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_15-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_15-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_15-5-16367c381d4b189b3640c92511244bfe | 1 - ...move_15-6-a4fb8359a2179ec70777aad6366071b7 | 1 - ...move_15-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_15-8-552c4eba867e7408fc8652ff0a19170d | 0 ...move_15-9-a63925fb2aa8c5df6854c248e674b0ef | 0 ...emove_16-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_16-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_16-10-a63925fb2aa8c5df6854c248e674b0ef | 0 ...ove_16-11-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_16-12-93b7341b523213ab6e58169459bc6818 | 0 ...ove_16-13-616cc477ed00e691dbc2b310d1c6dd12 | 0 ...move_16-14-ea111d286c70e4a0c6a68a7420dc7b7 | 32 - ...ove_16-15-37f9f7bc2d7456046a9f967347337e47 | 2 - ...ove_16-16-43d53504df013e6b35f81811138a167a | 1 - ...move_16-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_16-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_16-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_16-5-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_16-6-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_16-7-16367c381d4b189b3640c92511244bfe | 1 - ...move_16-8-a4fb8359a2179ec70777aad6366071b7 | 1 - ...move_16-9-ec47ba0fc527a4a04d452a009d59147a | 0 ...emove_17-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_17-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_17-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 | 0 ...ove_17-12-c91289e16ad403babfc91c093ac9b86d | 0 ...move_17-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 - ...ove_17-14-37f9f7bc2d7456046a9f967347337e47 | 2 - ...ove_17-15-43d53504df013e6b35f81811138a167a | 1 - ...ove_17-16-626a252f75285872c1d72706f7f972c6 | 6 - ...ove_17-17-1252985379f11ae4b98d2a0e2f121b8a | 6 - ...move_17-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_17-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_17-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_17-5-16367c381d4b189b3640c92511244bfe | 1 - ...move_17-6-a4fb8359a2179ec70777aad6366071b7 | 1 - ...move_17-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_17-8-67e57f56d1106a57329bf75c491e3c8b | 0 ...move_17-9-a63925fb2aa8c5df6854c248e674b0ef | 0 ...emove_18-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_18-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_18-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad | 0 ...ove_18-12-461a24ba6c00f60d75afba62da2ac4f1 | 0 ...move_18-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 - ...ove_18-14-37f9f7bc2d7456046a9f967347337e47 | 6 - ...ove_18-15-43d53504df013e6b35f81811138a167a | 1 - ...move_18-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_18-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_18-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_18-5-16367c381d4b189b3640c92511244bfe | 1 - ...move_18-6-a4fb8359a2179ec70777aad6366071b7 | 1 - ...move_18-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_18-8-b1b996b2e72ca14150af7c82dbc6e139 | 0 ...move_18-9-ea467d0fee062a23c720cf47eacfef08 | 0 ...emove_19-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_19-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_19-10-8d1e5af37e7992708bf15ab7d887405b | 0 ...move_19-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_19-12-68e15b1729669c4cb2291dcabfea4387 | 10 - ...ove_19-13-b8b9df8f376df228e3c8ae65defe2801 | 0 ...ove_19-14-eed866a1ad3106a306322519f4bb52f2 | 0 ...ove_19-15-68e15b1729669c4cb2291dcabfea4387 | 2 - ...ove_19-16-471f8e794fd712dce2e40334b383e08e | 0 ...ove_19-17-ae0f7d5734ca67cbfebed70c4657e330 | 0 ...ove_19-18-43d53504df013e6b35f81811138a167a | 1 - ...move_19-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_19-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_19-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_19-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_19-6-a6c043a89a9c3456af8ee065cb17239 | 0 ...move_19-7-48f70528347f5201f387d28dae37a14a | 0 ...move_19-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_19-9-8a08edd1aa63fd3b051da82246793259 | 0 ...remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_2-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_2-10-2309570010c3e679b884c100de57d002 | 0 ...emove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_2-12-43d53504df013e6b35f81811138a167a | 1 - ...emove_2-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_2-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_2-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_2-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...remove_2-6-1acf02741a2ff987d3e00ae9722c26e | 0 ...emove_2-7-48f70528347f5201f387d28dae37a14a | 0 ...emove_2-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...emove_20-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_20-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_20-10-89c57c91facbf54299e08955e3783ea6 | 0 ...move_20-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_20-12-43d53504df013e6b35f81811138a167a | 1 - ...move_20-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_20-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_20-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_20-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_20-6-a73143117ffec1225f6d492e5aa577e | 0 ...move_20-7-82f81adc097c247475fd29076e0cb85f | 0 ...move_20-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_20-9-683949ae07de12da0b7e7ba7f4450daa | 0 ...emove_21-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_21-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_21-10-e19469a1b70be25caaf670fe68f0a747 | 0 ...move_21-11-ea111d286c70e4a0c6a68a7420dc7b7 | 26 - ...ove_21-12-43d53504df013e6b35f81811138a167a | 1 - ...move_21-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_21-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_21-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_21-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_21-6-aeba356b56f8659963d8b2dc07a84a6f | 0 ...move_21-7-5716c408db679fb88352eaceb1703bd7 | 0 ...move_21-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_21-9-deadbce171926623b0586587fbbcd144 | 0 ...emove_22-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_22-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_22-10-403471d96c56b565272d2e4c4926d240 | 0 ...move_22-11-ea111d286c70e4a0c6a68a7420dc7b7 | 28 - ...ove_22-12-68e15b1729669c4cb2291dcabfea4387 | 10 - ...ove_22-13-a28b876b5df29140ef2bf62b4d0de3fd | 0 ...ove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af | 0 ...ove_22-15-43d53504df013e6b35f81811138a167a | 1 - ...move_22-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_22-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_22-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_22-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_22-6-722acd65700dd132dc6b4bc8c56f4ce0 | 0 ...move_22-7-46da090f5a2c73b175207cf63ff46653 | 0 ...move_22-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_22-9-b3edbbee0543ff268db4059afb9cb2cb | 0 ...emove_23-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_23-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_23-10-613ce50facecdc8d7bf8806a8ff17c13 | 0 ...move_23-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_23-12-43d53504df013e6b35f81811138a167a | 1 - ...move_23-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_23-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_23-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_23-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 | 0 ...move_23-7-48f70528347f5201f387d28dae37a14a | 0 ...move_23-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_23-9-2dca12ca51c74540e7cdbbb05e336ed5 | 0 ...emove_24-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_24-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_24-10-6d89089b1eead05510dbccad5fcc4805 | 0 ...move_24-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_24-12-43d53504df013e6b35f81811138a167a | 1 - ...move_24-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_24-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_24-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_24-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_24-6-72ae7a9640ae611f61ac954ac1a4b682 | 0 ...move_24-7-44a7b0f14b4f5151c37498367ad7fe1e | 0 ...move_24-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_24-9-4c0550cc9c28de25993c1f98de39168f | 0 ...remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 | 2 +- ...emove_3-1-16a6a293f1d2ce481b1d2482b1d5787c | 2 +- ...emove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 | 17 +- ...move_3-12-43d53504df013e6b35f81811138a167a | 2 +- ...move_3-13-e409e7032445097ace016b1876d95b3e | 2 +- ...emove_3-2-cafed8ca348b243372b9114910be1557 | 2 +- ...emove_3-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ...emove_3-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...emove_3-5-a2a411ad6620aa1ab24550ade336e785 | 2 +- ...emove_3-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_3-8-94da21f150ed2c56046b80e46da8884d} | 0 ...remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_4-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_4-10-19865a08066d80cb069ae6312c465ee6 | 0 ...move_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_4-13-43d53504df013e6b35f81811138a167a | 1 - ...emove_4-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_4-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...emove_4-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...emove_4-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_4-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...emove_4-7-90fcd4eb330919ad92aecb8a5bf30ead | 0 ...emove_4-8-48f70528347f5201f387d28dae37a14a | 0 ...emove_4-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_5-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_5-10-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...move_5-11-2309570010c3e679b884c100de57d002 | 0 ...emove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_5-13-43d53504df013e6b35f81811138a167a | 1 - ...emove_5-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_5-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...emove_5-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...emove_5-5-6f53d5613262d393d82d159ec5dc16dc | 1 - ...emove_5-6-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_5-7-4da6ca94af4caf4426e5676a32b70375 | 0 ...emove_5-8-48f70528347f5201f387d28dae37a14a | 0 ...emove_5-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 | 2 +- ...emove_6-1-16a6a293f1d2ce481b1d2482b1d5787c | 2 +- ...move_6-12-43d53504df013e6b35f81811138a167a | 2 +- ...move_6-13-e409e7032445097ace016b1876d95b3e | 2 +- ...move_6-14-f74b28904e86047150396bc42680ca38 | 2 +- ...emove_6-2-cafed8ca348b243372b9114910be1557 | 2 +- ...emove_6-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ...emove_6-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...emove_6-5-a2a411ad6620aa1ab24550ade336e785 | 2 +- ...emove_6-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_6-9-94da21f150ed2c56046b80e46da8884d} | 0 ...remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_7-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_7-12-43d53504df013e6b35f81811138a167a | 1 - ...emove_7-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_7-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_7-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_7-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_7-6-3744f0ebf5c002fdfcec67fbce03dfca | 0 ...emove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_7-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_7-9-19865a08066d80cb069ae6312c465ee6 | 0 ...remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_8-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_8-10-2309570010c3e679b884c100de57d002 | 0 ...emove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_8-12-43d53504df013e6b35f81811138a167a | 1 - ...emove_8-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_8-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_8-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_8-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_8-6-895f2432b4be6fcb11641c1d063570ee | 0 ...emove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_8-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_9-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_9-10-12cf3335c756f8715a07c5a604f10f64 | 0 ...move_9-11-4e84cd589eceda668833f8f19ec28e7c | 0 ...emove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_9-13-43d53504df013e6b35f81811138a167a | 1 - ...emove_9-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_9-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...emove_9-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...emove_9-5-6f53d5613262d393d82d159ec5dc16dc | 1 - ...emove_9-6-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_9-7-f77ac921b27860fac94cac6b352f3eb5 | 0 ...emove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_9-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ..._script-0-ca195b08d98d7f21fe93208499bf0ff6 | 2 +- ..._script-1-982cc6d7b98f8fb1055a10ef021e2769 | 2 +- ...on_view-0-e56367a21517656c18a5bcfeecb4327d | 0 ...on_view-1-c790d4344144460224b0f02be7e137a8 | 0 ...on_view-2-7e33b0744f57bdc8ebcd9d45348aef14 | 0 ...on_view-3-4a746bd076e063017c4d6a2f9218a6e4 | 0 ...on_view-4-d3d75f376f83b694b1dc62c46fa53f4e | 0 ...on_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 | 0 ...on_view-6-c9d7dcde469d3b9a66965a64dd15e4ae | 1 - ...on_view-7-3b03210f94ec40db9ab02620645014d1 | 1 - ...on_view-8-35f48c7d6fa164bb84643657bc9280a8 | 1 - ...rchar_2-3-a8c072e5b13997e9c79484b4af9d78da | 2 +- ...rchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f | 2 +- ...rchar_2-5-96353c24b5f2b361f72e5c26b4135519 | 2 +- ...rchar_2-6-2b62789d07b4044bc32190261bf3490f | 2 +- ..._join1-10-1958143ee083437e87662cadb48c37ce | 2 +- ...r_join1-11-a55f750032663f77066e4979dedea1c | 2 +- ...har_join1-6-6bb08c5baa913d9dc506aef65425ef | 0 ..._join1-6-ab4392aa5ff499ec43229425ff23e22f} | 0 ...r_join1-7-341118dab140b17f0b9d2c7b101d1298 | 0 ..._join1-7-a01639290aaceb4b85aa6e44319f6386} | 0 ...r_join1-8-afe7304d94450481c01ddbaf6cc3f596 | 0 ..._join1-8-c05d7b534b51cecdc2ba2de4ce57ba37} | 0 ...r_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 | 2 +- ...d_types-0-e41b0e9db7a9dbd0540e29df341933bc | 0 ...d_types-1-87a28b0e800f051525899324a064b878 | 0 ...d_types-10-92f25849990eb777ac2711f9dd2e628 | 1 - ..._types-11-9780781a92fdd992f3cee080a8717238 | 1 - ..._types-12-e1a4006971319a352280fc52eabf449f | 0 ..._types-13-d4574217a243a7d506398a819cd0eab4 | 0 ..._types-14-7c33a62195359bc89460ad65f6a5f763 | 1 - ...d_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 | 1 - ..._types-16-93811d146a429c44a2494b3aa4b2caa2 | 0 ..._types-17-5724af3985c67a0cb69919c9bbce15dc | 0 ..._types-18-d1be2ee3765a80469837ba11eb8685e1 | 1 - ..._types-19-a54f9a284228e7cdce8c34b8094f2377 | 1 - ...d_types-2-ae69b82461acc2aa366d8b1f8626d6fb | 0 ..._types-20-51a9279006f61097e68a52201daf6710 | 0 ..._types-21-1d304e09cc2a8b2824bfc04bdbc976ad | 1 - ..._types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 | 1 - ..._types-23-c7ea918777b725f2790da4fa00a3efa6 | 0 ..._types-24-5a69236334a3a1c4e771206cf547d730 | 1 - ..._types-25-61b030bb2220a533532d871ae0e08cdb | 1 - ..._types-26-e41b0e9db7a9dbd0540e29df341933bc | 0 ..._types-27-87a28b0e800f051525899324a064b878 | 0 ..._types-28-ae69b82461acc2aa366d8b1f8626d6fb | 0 ..._types-29-39d44d19c7963a9647fd3293eef670d4 | 0 ...d_types-3-39d44d19c7963a9647fd3293eef670d4 | 0 ..._types-30-162806477075d97de16dfa6f2576b751 | 0 ..._types-31-5b28e1fdb28b365ef419008a4752ed53 | 0 ...d_types-4-162806477075d97de16dfa6f2576b751 | 0 ...d_types-5-5b28e1fdb28b365ef419008a4752ed53 | 0 ...d_types-6-a47b99c355df4aad56dceb7f81a7fd5b | 0 ...d_types-7-8c483a7a0e148ca13a292a625f8702f1 | 0 ...d_types-8-94309e2b4e68ab7e25f7d9656f10b352 | 0 ...d_types-9-f248796769bc7f57cf56a75034a45520 | 0 ...r_serde-0-750a23ebdd77f32b555d4caba7ac5445 | 0 ...r_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d | 0 ..._serde-10-496280d9365ee601d9b68a91495d7160 | 0 ..._serde-11-2f4d5f574f659df2bd0fb97f2d50a36e | 0 ..._serde-12-80727f22f1343407ba9200c86ed84280 | 0 ..._serde-13-4829e6e5822c73fb33ba3d619b4bd31e | 5 - ..._serde-14-4794ee93811ce3a81f64c65aed0b8b13 | 5 - ..._serde-15-6ae634e1ae1aa0730a33396bce5a6604 | 0 ..._serde-16-516202183287d734d35d8c7788d22652 | 0 ..._serde-17-8d4419222a728e2bbc6a464b5a0b5f7a | 0 ..._serde-18-b2dfd28d6a5b1e6f4556ad19755b739d | 5 - ..._serde-19-f258df2db09e9cc0e049e85e6ad950ad | 5 - ...r_serde-2-87ba3f40293b9c79fcdb3064d964232e | 0 ..._serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 | 0 ..._serde-21-64536c77ae91bfb6cf7f93f178c6200b | 0 ...r_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 | 0 ..._serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 | 5 - ..._serde-24-770169b632b76cedcd6dfb87fdc46575 | 5 - ..._serde-25-3470a259b04e126c655531491787e2fc | 0 ..._serde-26-55808e190e0ab81dcdc1feb52543ad9f | 0 ..._serde-27-8fe526fdd347c25529a383f27ad20566 | 0 ..._serde-28-5e4de93349ba89a8344bb799ad60678e | 5 - ..._serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c | 5 - ...r_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 | 0 ..._serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a | 0 ..._serde-31-db1e88efcd55aaef567edaa89c1c1e12 | 0 ..._serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb | 0 ..._serde-33-620729dc7661c22f1acdc425a7cf0364 | 5 - ..._serde-34-807ee73e1cd66704dd585f7e0de954d9 | 5 - ..._serde-35-750a23ebdd77f32b555d4caba7ac5445 | 0 ..._serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d | 0 ..._serde-37-87ba3f40293b9c79fcdb3064d964232e | 0 ..._serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 | 0 ..._serde-39-8b5f81c27c41807c757522e257a8003f | 0 ...r_serde-4-8b5f81c27c41807c757522e257a8003f | 0 ..._serde-40-787193a1679a2153c037d3e4c8192bba | 0 ...r_serde-5-787193a1679a2153c037d3e4c8192bba | 0 ...r_serde-6-122f15d410249b554e12eccdfa46cc43 | 0 ...r_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 | 0 ...r_serde-8-3bfa13085b001c1a333cc72d5c9f4244 | 5 - ...r_serde-9-ec43be632e5f74057aba54c4f562c601 | 5 - ...union1-10-6ec48d5fea3e4a35275956b9b4467715 | 2 +- ...union1-11-78f6e219b974e1fdf3663e46f57892a9 | 2 +- ..._union1-6-67e66fa14dddc17757436539eca9ef64 | 0 ...union1-6-f338f341c5f86d0a44cabfb4f7bddc3b} | 0 ..._union1-7-48766d09c5ed1b6abe9ce0b8996adf36 | 0 ...union1-7-ea0d1fbae997b50dc34f7610480bbe29} | 0 ..._union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 | 0 ...union1-8-f3be9a2498927d692356c2cf871d25bf} | 0 ..._union1-9-b9455ffec62df97cfec63204ce02a110 | 2 +- .../view-0-5528e36b3b0f5b14313898cc45f9c23a | 0 .../view-1-7650b86c86dd6b1a99c86ddc5a31bd63 | 0 .../view-10-7aae4448a05e8a8a3bace7522e952cd0 | 0 .../view-11-dc95343d3e57846485dd543476391376 | 0 .../view-12-371764e1cae31ea0518c03060528d239 | 0 .../view-13-2abce88008f8a19164758ee821aaa8a6 | 0 .../view-14-deb504f4f70fd7db975950c3c47959ee | 0 .../view-15-6f2797b6f81943d3b53b8d247ae8512b | 0 .../view-2-9c529f486fa81a032bfe1253808fca8 | 0 .../view-3-89c80c0e90409d5e304775c9f420915a | 0 .../view-4-4a64d1a623ca71e515796787dbd0f904 | 0 .../view-5-f6d1bce095ecbf1aa484891392fdb07b | 0 .../view-6-47b5043f03a84695b6784682b4402ac8 | 0 .../view-7-8b1bbdadfd1e11af1b56064196164e58 | 0 .../view-8-60d2f3ee552ae7021f9fa72f0dcf2867 | 0 .../view-9-66c68babac10ae0f645fe8334c5a42d4 | 0 ...ew_cast-0-89bd46ad04f967f1f5ee17c6f201aacf | 0 ...ew_cast-1-85685a26971fb51ab6e28f9c5e8421bb | 0 ...w_cast-10-a7c865e5180df8d73dba90ede8be0d45 | 0 ...ew_cast-2-af2050aa97f0cd930cb1b8ec791007de | 0 ...iew_cast-3-2a232c31d056e6abc76f8ebe53ccd97 | 0 ...ew_cast-4-d9edb83f4cf847e141d97012314917d4 | 0 ...ew_cast-5-6db508ccd85562a9ca7841fb0a08981a | 0 ...ew_cast-6-aa5be3380ddc7104258567b406d93cc5 | 0 ...ew_cast-7-78ac3800b22682b31708b6a09b402bfb | 0 ...ew_cast-8-2cc0c576f0a008abf5bdf3308d500869 | 0 ...ew_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 | 0 ..._inputs-0-9e67dfd1d595ab8b1935b789645f76c0 | 0 ..._inputs-1-5af97e73bc3841793440105aae766bbe | 0 ..._column-0-9bacd1908e56d621913a74fe9a583d9d | 500 ---- ..._column-1-3ebad682d4ff6ca9c806db3471cf3945 | 309 -- ...column-10-2915b222a58bc994246591e536d388b4 | 0 ..._column-2-1536b365fe0a94b30a62364996529966 | 309 -- ..._column-3-c66776673c986b59b27e704664935988 | 0 ..._column-4-e47094c927b1091e31c185db0a4e69a6 | 1 - ..._column-5-d137fa7c27bc98d5f1a33f666a07f6b7 | 0 ..._column-6-68d6973677af5c9f1f1f49360c3175e7 | 1 - ..._column-7-20d7d672a4289fbd1a5de485a8353ac6 | 0 ..._column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 | 500 ---- ...l_column-9-c152da33c1517ecfc336f72b9c133d5 | 0 sql/hive/src/test/resources/log4j.properties | 6 + .../clientcompare/vectorized_math_funcs.q | 43 + .../clientcompare/vectorized_math_funcs_00.qv | 1 + .../clientcompare/vectorized_math_funcs_01.qv | 1 + .../alter_concatenate_indexed_table.q | 6 +- .../alter_partition_invalidspec.q | 4 +- .../clientnegative/alter_partition_nodrop.q | 4 +- .../alter_partition_nodrop_table.q | 4 +- .../clientnegative/alter_partition_offline.q | 4 +- .../alter_rename_partition_failure.q | 2 +- .../alter_rename_partition_failure2.q | 2 +- .../alter_rename_partition_failure3.q | 2 +- .../queries/clientnegative/ambiguous_col1.q | 1 + .../queries/clientnegative/ambiguous_col2.q | 1 + .../clientnegative/analyze_non_existent_tbl.q | 1 + .../queries/clientnegative/archive_corrupt.q | 2 +- .../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 + .../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 + .../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 + .../clientnegative/bucket_mapjoin_mismatch1.q | 10 +- .../bucket_mapjoin_wrong_table_metadata_1.q | 6 +- .../bucket_mapjoin_wrong_table_metadata_2.q | 10 +- .../test/queries/clientnegative/clustern1.q | 2 - .../clientnegative/columnstats_partlvl_dp.q | 12 +- .../columnstats_partlvl_incorrect_num_keys.q | 12 +- .../columnstats_partlvl_invalid_values.q | 12 +- ...columnstats_partlvl_multiple_part_clause.q | 12 +- .../clientnegative/columnstats_tbllvl.q | 2 +- .../columnstats_tbllvl_complex_type.q | 2 +- .../columnstats_tbllvl_incorrect_column.q | 2 +- .../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 + .../queries/clientnegative/cte_recursion.q | 4 + .../clientnegative/cte_with_in_subquery.q | 1 + .../queries/clientnegative/date_literal1.q | 2 - .../clientnegative/dbtxnmgr_nodblock.q | 6 + .../clientnegative/dbtxnmgr_nodbunlock.q | 6 + .../clientnegative/dbtxnmgr_notablelock.q | 6 + .../clientnegative/dbtxnmgr_notableunlock.q | 6 + .../test/queries/clientnegative/deletejar.q | 4 +- .../clientnegative/drop_func_nonexistent.q | 3 + .../drop_partition_filter_failure2.q | 11 - .../dynamic_partitions_with_whitelist.q | 2 +- ...hange_partition_neg_incomplete_partition.q | 4 +- .../exchange_partition_neg_partition_exists.q | 2 +- ...exchange_partition_neg_partition_exists2.q | 2 +- ...exchange_partition_neg_partition_exists3.q | 2 +- ...exchange_partition_neg_partition_missing.q | 2 +- .../exim_00_unsupported_schema.q | 6 +- .../exim_01_nonpart_over_loaded.q | 10 +- .../exim_02_all_part_over_overlap.q | 16 +- .../exim_03_nonpart_noncompat_colschema.q | 8 +- .../exim_04_nonpart_noncompat_colnumber.q | 8 +- .../exim_05_nonpart_noncompat_coltype.q | 8 +- .../exim_06_nonpart_noncompat_storage.q | 8 +- .../exim_07_nonpart_noncompat_ifof.q | 8 +- .../exim_08_nonpart_noncompat_serde.q | 8 +- .../exim_09_nonpart_noncompat_serdeparam.q | 8 +- .../exim_10_nonpart_noncompat_bucketing.q | 8 +- .../exim_11_nonpart_noncompat_sorting.q | 8 +- .../clientnegative/exim_13_nonnative_import.q | 8 +- .../clientnegative/exim_14_nonpart_part.q | 8 +- .../clientnegative/exim_15_part_nonpart.q | 8 +- .../exim_16_part_noncompat_schema.q | 8 +- .../exim_17_part_spec_underspec.q | 14 +- .../exim_18_part_spec_missing.q | 14 +- .../exim_19_external_over_existing.q | 8 +- .../exim_20_managed_location_over_existing.q | 14 +- .../exim_21_part_managed_external.q | 14 +- .../clientnegative/exim_22_export_authfail.q | 4 +- .../exim_23_import_exist_authfail.q | 8 +- .../exim_24_import_part_authfail.q | 8 +- .../exim_25_import_nonexist_authfail.q | 8 +- .../clientnegative/fetchtask_ioexception.q | 2 +- .../file_with_header_footer_negative.q | 13 + .../clientnegative/illegal_partition_type.q | 2 +- .../clientnegative/illegal_partition_type3.q | 4 + .../clientnegative/illegal_partition_type4.q | 3 + .../index_compact_entry_limit.q | 1 + .../clientnegative/index_compact_size_limit.q | 1 + .../queries/clientnegative/insert_into5.q | 9 + .../queries/clientnegative/insert_into6.q | 17 + .../insertover_dynapart_ifnotexists.q | 2 +- .../clientnegative/invalid_char_length_1.q | 2 + .../clientnegative/invalid_char_length_2.q | 1 + .../clientnegative/invalid_char_length_3.q | 3 + .../queries/clientnegative/invalid_columns.q | 4 - .../clientnegative/join_alt_syntax_comma_on.q | 3 + .../join_cond_unqual_ambiguous.q | 6 + .../join_cond_unqual_ambiguous_vc.q | 5 + .../queries/clientnegative/limit_partition.q | 7 + .../clientnegative/limit_partition_stats.q | 18 + .../clientnegative/load_exist_part_authfail.q | 2 +- .../queries/clientnegative/load_non_native.q | 2 +- .../clientnegative/load_nonpart_authfail.q | 2 +- .../clientnegative/load_part_authfail.q | 2 +- .../queries/clientnegative/load_part_nospec.q | 2 +- .../clientnegative/load_stored_as_dirs.q | 2 +- .../clientnegative/load_view_failure.q | 2 +- .../clientnegative/load_wrong_fileformat.q | 2 +- .../load_wrong_fileformat_rc_seq.q | 2 +- .../load_wrong_fileformat_txt_seq.q | 2 +- .../clientnegative/load_wrong_noof_part.q | 2 +- .../clientnegative/local_mapred_error_cache.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/nested_complex_neg.q | 2 +- .../queries/clientnegative/nopart_insert.q | 2 +- .../test/queries/clientnegative/nopart_load.q | 2 +- .../queries/clientnegative/notable_alias3.q | 4 - .../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/protectmode_part2.q | 4 +- ...ptf_negative_AggrFuncsWithNoGBYNoPartDef.q | 2 +- .../ptf_negative_AmbiguousWindowDefn.q | 2 +- .../test/queries/clientnegative/regex_col_1.q | 1 + .../test/queries/clientnegative/regex_col_2.q | 1 + .../clientnegative/regex_col_groupby.q | 1 + .../clientnegative/script_broken_pipe1.q | 3 - .../queries/clientnegative/script_error.q | 4 +- .../queries/clientnegative/serde_regex2.q | 4 +- .../clientnegative/set_hiveconf_validation2.q | 5 + .../clientnegative/stats_aggregator_error_1.q | 2 +- .../clientnegative/stats_aggregator_error_2.q | 2 +- .../clientnegative/stats_publisher_error_1.q | 2 +- .../clientnegative/stats_publisher_error_2.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 + .../clientnegative/udf_case_type_wrong.q | 6 - .../clientnegative/udf_case_type_wrong2.q | 6 - .../clientnegative/udf_case_type_wrong3.q | 6 - .../test/queries/clientnegative/udf_invalid.q | 1 + .../clientnegative/udf_local_resource.q | 1 + .../clientnegative/udf_nonexistent_resource.q | 1 + .../clientnegative/udf_qualified_name.q | 1 + .../clientnegative/udf_when_type_wrong2.q | 6 - .../clientnegative/udf_when_type_wrong3.q | 6 - .../src/test/queries/clientnegative/udfnull.q | 6 - .../src/test/queries/clientnegative/union.q | 4 - .../clientnegative/windowing_invalid_udaf.q | 1 + .../clientnegative/windowing_ll_no_neg.q | 26 + .../src/test/queries/clientpositive/alter1.q | 4 +- .../src/test/queries/clientpositive/alter3.q | 4 +- .../src/test/queries/clientpositive/alter5.q | 4 +- .../test/queries/clientpositive/alter_char1.q | 32 + .../test/queries/clientpositive/alter_char2.q | 22 + .../alter_concatenate_indexed_table.q | 12 +- .../queries/clientpositive/alter_db_owner.q | 9 + .../test/queries/clientpositive/alter_merge.q | 12 +- .../queries/clientpositive/alter_merge_2.q | 6 +- .../clientpositive/alter_merge_stats.q | 12 +- .../alter_numbuckets_partitioned_table.q | 2 +- .../alter_numbuckets_partitioned_table2.q | 1 + .../alter_numbuckets_partitioned_table2_h23.q | 85 + .../alter_numbuckets_partitioned_table_h23.q | 59 + .../clientpositive/alter_partition_coltype.q | 42 +- .../alter_partition_protect_mode.q | 8 +- .../clientpositive/alter_rename_partition.q | 4 +- .../queries/clientpositive/alter_varchar2.q | 6 +- .../queries/clientpositive/ambiguous_col.q | 1 + .../clientpositive/annotate_stats_filter.q | 76 + .../clientpositive/annotate_stats_groupby.q | 69 + .../clientpositive/annotate_stats_join.q | 81 + .../clientpositive/annotate_stats_limit.q | 30 + .../clientpositive/annotate_stats_part.q | 85 + .../clientpositive/annotate_stats_select.q | 143 + .../clientpositive/annotate_stats_table.q | 53 + .../clientpositive/annotate_stats_union.q | 55 + .../clientpositive/ansi_sql_arithmetic.q | 13 + .../queries/clientpositive/archive_corrupt.q | 2 +- .../clientpositive/archive_excludeHadoop20.q | 2 +- .../clientpositive/authorization_1_sql_std.q | 36 + .../queries/clientpositive/authorization_2.q | 2 +- .../queries/clientpositive/authorization_9.q | 17 + .../authorization_admin_almighty1.q | 17 + .../authorization_create_func1.q | 14 + .../authorization_create_macro1.q | 12 + .../authorization_create_table_owner_privs.q | 10 + .../authorization_grant_public_role.q | 18 + .../authorization_grant_table_priv.q | 43 + .../clientpositive/authorization_index.q | 12 + .../authorization_owner_actions.q | 16 + .../authorization_owner_actions_db.q | 21 + .../clientpositive/authorization_parts.q | 19 + .../authorization_revoke_table_priv.q | 61 + .../authorization_role_grant1.q | 38 + .../authorization_role_grant2.q | 34 + .../authorization_set_show_current_role.q | 21 + .../authorization_view_sqlstd.q | 66 + .../test/queries/clientpositive/auto_join25.q | 2 + .../test/queries/clientpositive/auto_join32.q | 16 +- .../clientpositive/auto_join_filters.q | 10 +- .../queries/clientpositive/auto_join_nulls.q | 2 +- .../auto_join_reordering_values.q | 6 +- .../auto_join_without_localtask.q | 29 + .../clientpositive/auto_sortmerge_join_1.q | 22 +- .../clientpositive/auto_sortmerge_join_11.q | 22 +- .../clientpositive/auto_sortmerge_join_12.q | 26 +- .../clientpositive/auto_sortmerge_join_16.q | 92 + .../clientpositive/auto_sortmerge_join_2.q | 16 +- .../clientpositive/auto_sortmerge_join_3.q | 16 +- .../clientpositive/auto_sortmerge_join_4.q | 20 +- .../clientpositive/auto_sortmerge_join_5.q | 12 +- .../clientpositive/auto_sortmerge_join_7.q | 24 +- .../clientpositive/auto_sortmerge_join_8.q | 26 +- .../clientpositive/avro_compression_enabled.q | 2 +- .../clientpositive/avro_evolved_schemas.q | 2 +- .../test/queries/clientpositive/avro_joins.q | 4 +- .../clientpositive/avro_nullable_fields.q | 2 +- .../queries/clientpositive/avro_partitioned.q | 4 +- .../queries/clientpositive/avro_sanity_test.q | 2 +- .../queries/clientpositive/binary_constant.q | 4 +- .../clientpositive/binary_table_colserde.q | 1 + .../queries/clientpositive/binarysortable_1.q | 2 +- .../bucket_if_with_path_filter.q | 15 + .../clientpositive/bucket_map_join_1.q | 4 +- .../clientpositive/bucket_map_join_2.q | 4 +- .../clientpositive/bucket_map_join_tez1.q | 85 + .../clientpositive/bucket_map_join_tez2.q | 50 + .../queries/clientpositive/bucketcontext_1.q | 20 +- .../queries/clientpositive/bucketcontext_2.q | 16 +- .../queries/clientpositive/bucketcontext_3.q | 16 +- .../queries/clientpositive/bucketcontext_4.q | 20 +- .../queries/clientpositive/bucketcontext_5.q | 12 +- .../queries/clientpositive/bucketcontext_6.q | 16 +- .../queries/clientpositive/bucketcontext_7.q | 24 +- .../queries/clientpositive/bucketcontext_8.q | 24 +- .../bucketizedhiveinputformat.q | 6 +- .../bucketizedhiveinputformat_auto.q | 20 +- .../queries/clientpositive/bucketmapjoin1.q | 16 +- .../queries/clientpositive/bucketmapjoin10.q | 20 +- .../queries/clientpositive/bucketmapjoin11.q | 24 +- .../queries/clientpositive/bucketmapjoin12.q | 12 +- .../queries/clientpositive/bucketmapjoin2.q | 16 +- .../queries/clientpositive/bucketmapjoin3.q | 16 +- .../queries/clientpositive/bucketmapjoin4.q | 16 +- .../queries/clientpositive/bucketmapjoin5.q | 28 +- .../queries/clientpositive/bucketmapjoin7.q | 8 +- .../queries/clientpositive/bucketmapjoin8.q | 8 +- .../queries/clientpositive/bucketmapjoin9.q | 14 +- .../clientpositive/bucketmapjoin_negative.q | 10 +- .../clientpositive/bucketmapjoin_negative2.q | 12 +- .../clientpositive/bucketmapjoin_negative3.q | 24 +- .../test/queries/clientpositive/cast_to_int.q | 4 +- .../src/test/queries/clientpositive/char_1.q | 32 + .../src/test/queries/clientpositive/char_2.q | 36 + .../test/queries/clientpositive/char_cast.q | 92 + .../queries/clientpositive/char_comparison.q | 40 + .../test/queries/clientpositive/char_join1.q | 35 + .../clientpositive/char_nested_types.q | 53 + .../test/queries/clientpositive/char_serde.q | 102 + .../test/queries/clientpositive/char_udf1.q | 156 + .../test/queries/clientpositive/char_union1.q | 47 + .../queries/clientpositive/char_varchar_udf.q | 9 + .../clientpositive/column_access_stats.q | 2 +- .../clientpositive/columnstats_partlvl.q | 4 +- .../clientpositive/columnstats_tbllvl.q | 2 +- .../clientpositive/compile_processor.q | 12 + .../clientpositive/compute_stats_binary.q | 2 +- .../clientpositive/compute_stats_boolean.q | 2 +- .../clientpositive/compute_stats_decimal.q | 11 + .../clientpositive/compute_stats_double.q | 2 +- .../clientpositive/compute_stats_long.q | 2 +- .../clientpositive/compute_stats_string.q | 2 +- .../queries/clientpositive/constant_prop.q | 6 +- .../clientpositive/correlationoptimizer1.q | 25 +- .../clientpositive/correlationoptimizer4.q | 6 +- .../clientpositive/correlationoptimizer5.q | 8 +- .../src/test/queries/clientpositive/count.q | 2 +- .../queries/clientpositive/create_func1.q | 30 + .../test/queries/clientpositive/create_like.q | 6 +- .../clientpositive/create_merge_compressed.q | 2 +- .../clientpositive/create_nested_type.q | 2 +- .../clientpositive/create_struct_table.q | 2 +- .../clientpositive/create_union_table.q | 2 +- .../clientpositive/create_view_translate.q | 11 + .../clientpositive/cross_product_check_1.q | 26 + .../clientpositive/cross_product_check_2.q | 27 + .../ql/src/test/queries/clientpositive/ctas.q | 2 +- .../test/queries/clientpositive/ctas_char.q | 22 + .../queries/clientpositive/ctas_hadoop20.q | 11 +- .../src/test/queries/clientpositive/cte_1.q | 28 + .../src/test/queries/clientpositive/cte_2.q | 56 + .../custom_input_output_format.q | 5 +- .../test/queries/clientpositive/database.q | 6 +- .../queries/clientpositive/database_drop.q | 17 +- .../src/test/queries/clientpositive/date_1.q | 8 +- .../src/test/queries/clientpositive/date_2.q | 2 +- .../src/test/queries/clientpositive/date_3.q | 2 +- .../src/test/queries/clientpositive/date_4.q | 4 +- .../queries/clientpositive/date_comparison.q | 2 + .../test/queries/clientpositive/date_join1.q | 2 +- .../test/queries/clientpositive/date_serde.q | 2 +- .../test/queries/clientpositive/date_udf.q | 2 +- .../clientpositive/dbtxnmgr_compact1.q | 12 + .../clientpositive/dbtxnmgr_compact2.q | 14 + .../clientpositive/dbtxnmgr_compact3.q | 15 + .../queries/clientpositive/dbtxnmgr_ddl1.q | 59 + .../queries/clientpositive/dbtxnmgr_query1.q | 17 + .../queries/clientpositive/dbtxnmgr_query2.q | 17 + .../queries/clientpositive/dbtxnmgr_query3.q | 21 + .../queries/clientpositive/dbtxnmgr_query4.q | 19 + .../queries/clientpositive/dbtxnmgr_query5.q | 24 + .../clientpositive/dbtxnmgr_showlocks.q | 11 + .../test/queries/clientpositive/decimal_1.q | 28 +- .../test/queries/clientpositive/decimal_2.q | 60 +- .../test/queries/clientpositive/decimal_3.q | 4 +- .../test/queries/clientpositive/decimal_4.q | 6 +- .../test/queries/clientpositive/decimal_5.q | 18 + .../test/queries/clientpositive/decimal_6.q | 27 + .../queries/clientpositive/decimal_join.q | 4 +- .../clientpositive/decimal_precision.q | 15 +- .../queries/clientpositive/decimal_serde.q | 2 +- .../test/queries/clientpositive/decimal_udf.q | 10 +- .../test/queries/clientpositive/delimiter.q | 2 +- .../clientpositive/desc_tbl_part_cols.q | 7 + .../disable_file_format_check.q | 4 +- .../disallow_incompatible_type_change_off.q | 4 +- .../queries/clientpositive/distinct_stats.q | 20 + .../clientpositive/drop_partitions_filter2.q | 5 +- .../clientpositive/drop_with_concurrency.q | 8 + .../dynamic_partition_skip_default.q | 16 +- .../dynpart_sort_opt_vectorization.q | 161 ++ .../dynpart_sort_optimization.q | 155 + .../src/test/queries/clientpositive/escape1.q | 2 +- .../src/test/queries/clientpositive/escape2.q | 2 +- .../clientpositive/exchange_partition.q | 2 +- .../clientpositive/exchange_partition2.q | 2 +- .../clientpositive/exchange_partition3.q | 5 +- .../clientpositive/exim_00_nonpart_empty.q | 9 +- .../queries/clientpositive/exim_01_nonpart.q | 8 +- .../clientpositive/exim_02_00_part_empty.q | 6 +- .../queries/clientpositive/exim_02_part.q | 8 +- .../exim_03_nonpart_over_compat.q | 8 +- .../queries/clientpositive/exim_04_all_part.q | 14 +- .../clientpositive/exim_04_evolved_parts.q | 6 +- .../clientpositive/exim_05_some_part.q | 14 +- .../queries/clientpositive/exim_06_one_part.q | 14 +- .../exim_07_all_part_over_nonoverlap.q | 16 +- .../clientpositive/exim_08_nonpart_rename.q | 10 +- .../exim_09_part_spec_nonoverlap.q | 18 +- .../clientpositive/exim_10_external_managed.q | 14 +- .../clientpositive/exim_11_managed_external.q | 8 +- .../exim_12_external_location.q | 14 +- .../clientpositive/exim_13_managed_location.q | 14 +- .../exim_14_managed_location_over_existing.q | 14 +- .../clientpositive/exim_15_external_part.q | 24 +- .../clientpositive/exim_16_part_external.q | 26 +- .../clientpositive/exim_17_part_managed.q | 20 +- .../clientpositive/exim_18_part_external.q | 14 +- .../exim_19_00_part_external_location.q | 16 +- .../exim_19_part_external_location.q | 20 +- .../exim_20_part_managed_location.q | 20 +- .../exim_21_export_authsuccess.q | 6 +- .../exim_22_import_exist_authsuccess.q | 8 +- .../exim_23_import_part_authsuccess.q | 8 +- .../exim_24_import_nonexist_authsuccess.q | 8 +- .../clientpositive/exim_hidden_files.q | 22 + .../clientpositive/explain_rearrange.q | 98 + ...ternal_table_with_space_in_location_path.q | 23 + .../clientpositive/file_with_header_footer.q | 39 + .../clientpositive/filter_join_breaktask2.q | 6 +- .../queries/clientpositive/filter_numeric.q | 21 + .../queries/clientpositive/global_limit.q | 14 +- .../test/queries/clientpositive/groupby10.q | 2 +- .../test/queries/clientpositive/groupby12.q | 13 + .../queries/clientpositive/groupby1_limit.q | 2 +- .../queries/clientpositive/groupby1_map.q | 2 +- .../clientpositive/groupby1_map_skew.q | 2 +- .../queries/clientpositive/groupby1_noskew.q | 2 +- .../queries/clientpositive/groupby2_limit.q | 4 +- .../queries/clientpositive/groupby2_map.q | 2 +- .../groupby2_map_multi_distinct.q | 13 +- .../queries/clientpositive/groupby2_noskew.q | 2 +- .../groupby2_noskew_multi_distinct.q | 2 +- .../queries/clientpositive/groupby4_noskew.q | 2 +- .../queries/clientpositive/groupby5_noskew.q | 2 +- .../queries/clientpositive/groupby6_map.q | 2 +- .../clientpositive/groupby6_map_skew.q | 2 +- .../queries/clientpositive/groupby6_noskew.q | 2 +- .../queries/clientpositive/groupby7_map.q | 4 +- .../groupby7_map_multi_single_reducer.q | 4 +- .../clientpositive/groupby7_map_skew.q | 4 +- .../queries/clientpositive/groupby7_noskew.q | 4 +- .../groupby7_noskew_multi_single_reducer.q | 4 +- .../queries/clientpositive/groupby8_map.q | 4 +- .../clientpositive/groupby8_map_skew.q | 4 +- .../queries/clientpositive/groupby8_noskew.q | 4 +- .../queries/clientpositive/groupby_bigdata.q | 2 +- .../queries/clientpositive/groupby_cube1.q | 2 +- .../clientpositive/groupby_grouping_id1.q | 2 +- .../clientpositive/groupby_grouping_id2.q | 2 +- .../clientpositive/groupby_grouping_sets1.q | 2 +- .../clientpositive/groupby_grouping_sets2.q | 2 +- .../clientpositive/groupby_grouping_sets3.q | 4 +- .../clientpositive/groupby_grouping_sets4.q | 2 +- .../clientpositive/groupby_grouping_sets5.q | 2 +- .../queries/clientpositive/groupby_map_ppr.q | 2 +- .../groupby_map_ppr_multi_distinct.q | 2 +- .../clientpositive/groupby_resolution.q | 61 + .../queries/clientpositive/groupby_rollup1.q | 2 +- .../queries/clientpositive/groupby_sort_1.q | 2 +- .../queries/clientpositive/groupby_sort_2.q | 2 +- .../queries/clientpositive/groupby_sort_3.q | 2 +- .../queries/clientpositive/groupby_sort_4.q | 2 +- .../queries/clientpositive/groupby_sort_5.q | 6 +- .../queries/clientpositive/groupby_sort_6.q | 2 +- .../queries/clientpositive/groupby_sort_7.q | 2 +- .../queries/clientpositive/groupby_sort_8.q | 2 +- .../queries/clientpositive/groupby_sort_9.q | 2 +- .../clientpositive/groupby_sort_skew_1.q | 2 +- .../clientpositive/groupby_sort_test_1.q | 2 +- .../clientpositive/import_exported_table.q | 13 + .../test/queries/clientpositive/index_auth.q | 13 +- .../test/queries/clientpositive/index_auto.q | 3 +- .../queries/clientpositive/index_auto_empty.q | 1 + .../clientpositive/index_auto_file_format.q | 3 +- .../clientpositive/index_auto_mult_tables.q | 1 + .../index_auto_mult_tables_compact.q | 1 + .../clientpositive/index_auto_multiple.q | 3 +- .../clientpositive/index_auto_partitioned.q | 1 + .../clientpositive/index_auto_self_join.q | 1 + .../clientpositive/index_auto_unused.q | 1 + .../queries/clientpositive/index_bitmap.q | 1 + .../queries/clientpositive/index_bitmap1.q | 1 + .../queries/clientpositive/index_bitmap2.q | 1 + .../queries/clientpositive/index_bitmap3.q | 11 +- .../clientpositive/index_bitmap_auto.q | 15 +- .../index_bitmap_auto_partitioned.q | 1 + .../clientpositive/index_bitmap_compression.q | 1 + .../queries/clientpositive/index_bitmap_rc.q | 1 + .../queries/clientpositive/index_compact.q | 1 + .../queries/clientpositive/index_compact_1.q | 1 + .../queries/clientpositive/index_compact_2.q | 3 +- .../queries/clientpositive/index_compact_3.q | 3 +- .../index_compact_binary_search.q | 4 +- .../clientpositive/index_compression.q | 1 + .../queries/clientpositive/index_creation.q | 1 + .../test/queries/clientpositive/index_serde.q | 3 +- .../test/queries/clientpositive/index_stale.q | 1 + .../clientpositive/index_stale_partitioned.q | 1 + .../infer_bucket_sort_dyn_part.q | 10 +- .../queries/clientpositive/infer_const_type.q | 2 +- .../src/test/queries/clientpositive/input13.q | 6 +- .../src/test/queries/clientpositive/input16.q | 4 +- .../test/queries/clientpositive/input16_cc.q | 4 +- .../src/test/queries/clientpositive/input19.q | 2 +- .../src/test/queries/clientpositive/input20.q | 6 +- .../src/test/queries/clientpositive/input21.q | 2 +- .../src/test/queries/clientpositive/input22.q | 2 +- .../src/test/queries/clientpositive/input33.q | 6 +- .../src/test/queries/clientpositive/input37.q | 4 +- .../queries/clientpositive/input3_limit.q | 4 +- .../src/test/queries/clientpositive/input4.q | 4 +- .../src/test/queries/clientpositive/input40.q | 6 +- .../src/test/queries/clientpositive/input43.q | 4 +- .../src/test/queries/clientpositive/input44.q | 2 +- .../src/test/queries/clientpositive/input45.q | 6 +- .../queries/clientpositive/input4_cb_delim.q | 2 +- .../test/queries/clientpositive/input_dfs.q | 2 +- .../test/queries/clientpositive/inputddl5.q | 2 +- .../test/queries/clientpositive/inputddl6.q | 4 +- .../test/queries/clientpositive/inputddl7.q | 8 +- .../insert1_overwrite_partitions.q | 4 +- .../insert2_overwrite_partitions.q | 4 +- .../queries/clientpositive/insert_into3.q | 8 +- .../insert_overwrite_local_directory_1.q | 52 +- .../test/queries/clientpositive/join_1to1.q | 4 +- .../queries/clientpositive/join_alt_syntax.q | 41 + .../test/queries/clientpositive/join_array.q | 4 +- .../clientpositive/join_casesensitive.q | 4 +- .../clientpositive/join_cond_pushdown_1.q | 30 + .../clientpositive/join_cond_pushdown_2.q | 24 + .../clientpositive/join_cond_pushdown_3.q | 34 + .../clientpositive/join_cond_pushdown_4.q | 26 + .../join_cond_pushdown_unqual1.q | 52 + .../join_cond_pushdown_unqual2.q | 47 + .../join_cond_pushdown_unqual3.q | 56 + .../join_cond_pushdown_unqual4.q | 49 + .../queries/clientpositive/join_filters.q | 10 +- .../queries/clientpositive/join_hive_626.q | 6 +- .../queries/clientpositive/join_merging.q | 25 + .../test/queries/clientpositive/join_nulls.q | 10 +- .../queries/clientpositive/join_nullsafe.q | 10 +- .../queries/clientpositive/join_reorder.q | 6 +- .../queries/clientpositive/join_reorder2.q | 8 +- .../queries/clientpositive/join_reorder3.q | 8 +- .../queries/clientpositive/join_reorder4.q | 6 +- .../test/queries/clientpositive/join_star.q | 16 +- .../clientpositive/lateral_view_noalias.q | 2 + .../queries/clientpositive/lateral_view_ppd.q | 4 + .../test/queries/clientpositive/lb_fs_stats.q | 19 + .../src/test/queries/clientpositive/leadlag.q | 2 +- .../queries/clientpositive/leadlag_queries.q | 2 +- .../queries/clientpositive/leftsemijoin.q | 6 +- .../queries/clientpositive/leftsemijoin_mr.q | 4 +- .../limit_partition_metadataonly.q | 7 + .../queries/clientpositive/limit_pushdown.q | 13 +- .../clientpositive/limit_pushdown_negative.q | 4 - .../clientpositive/list_bucket_dml_2.q | 6 +- .../clientpositive/list_bucket_dml_4.q | 2 +- .../queries/clientpositive/literal_decimal.q | 2 + .../queries/clientpositive/literal_double.q | 2 + .../queries/clientpositive/literal_ints.q | 2 + .../queries/clientpositive/literal_string.q | 2 + .../queries/clientpositive/load_binary_data.q | 2 +- .../queries/clientpositive/load_dyn_part1.q | 4 +- .../queries/clientpositive/load_dyn_part10.q | 2 +- .../queries/clientpositive/load_dyn_part3.q | 2 +- .../queries/clientpositive/load_dyn_part4.q | 4 +- .../queries/clientpositive/load_dyn_part8.q | 2 +- .../queries/clientpositive/load_dyn_part9.q | 2 +- .../load_exist_part_authsuccess.q | 2 +- .../load_file_with_space_in_the_name.q | 3 +- .../src/test/queries/clientpositive/load_fs.q | 6 +- .../test/queries/clientpositive/load_fs2.q | 6 +- .../clientpositive/load_fs_overwrite.q | 20 + .../load_hdfs_file_with_space_in_the_name.q | 9 +- .../clientpositive/load_nonpart_authsuccess.q | 2 +- .../queries/clientpositive/load_overwrite.q | 4 +- .../clientpositive/load_part_authsuccess.q | 2 +- .../test/queries/clientpositive/loadpart1.q | 2 +- .../test/queries/clientpositive/loadpart2.q | 9 + .../queries/clientpositive/loadpart_err.q | 4 +- .../src/test/queries/clientpositive/macro.q | 2 + .../queries/clientpositive/mapjoin_addjar.q | 14 + .../queries/clientpositive/mapjoin_decimal.q | 35 + .../queries/clientpositive/mapjoin_hook.q | 3 +- .../queries/clientpositive/mapjoin_mapjoin.q | 10 +- .../queries/clientpositive/mapjoin_memcheck.q | 16 + .../clientpositive/mapjoin_subquery2.q | 6 +- .../src/test/queries/clientpositive/merge3.q | 4 +- .../src/test/queries/clientpositive/merge4.q | 4 +- .../clientpositive/merge_dynamic_partition.q | 9 +- .../clientpositive/merge_dynamic_partition2.q | 13 +- .../clientpositive/merge_dynamic_partition3.q | 28 +- .../clientpositive/merge_dynamic_partition4.q | 18 +- .../clientpositive/merge_dynamic_partition5.q | 10 +- .../clientpositive/metadata_export_drop.q | 6 +- .../clientpositive/metadata_only_queries.q | 77 + .../metadata_only_queries_with_filters.q | 51 + .../ql/src/test/queries/clientpositive/mi.q | 4 +- .../ql/src/test/queries/clientpositive/mrr.q | 59 + .../queries/clientpositive/multiMapJoin1.q | 2 + .../queries/clientpositive/multiMapJoin2.q | 26 + .../queries/clientpositive/nested_complex.q | 2 +- .../src/test/queries/clientpositive/newline.q | 10 +- .../clientpositive/nonmr_fetch_threshold.q | 9 + .../nonreserved_keywords_input37.q | 4 +- .../queries/clientpositive/notable_alias3.q | 4 + .../test/queries/clientpositive/null_cast.q | 4 +- .../test/queries/clientpositive/null_column.q | 6 +- .../test/queries/clientpositive/nullformat.q | 24 + .../queries/clientpositive/nullformatCTAS.q | 24 + .../queries/clientpositive/nullformatdir.q | 21 + .../test/queries/clientpositive/nullgroup3.q | 16 +- .../test/queries/clientpositive/nullgroup5.q | 4 +- .../test/queries/clientpositive/nullscript.q | 4 +- .../queries/clientpositive/num_op_type_conv.q | 2 + .../queries/clientpositive/ops_comparison.q | 1 + .../queries/clientpositive/optrstat_groupby.q | 6 - .../test/queries/clientpositive/orc_analyze.q | 179 ++ .../test/queries/clientpositive/orc_create.q | 31 +- .../clientpositive/orc_dictionary_threshold.q | 2 +- .../clientpositive/orc_diff_part_cols.q | 2 +- .../clientpositive/orc_diff_part_cols2.q | 11 + .../clientpositive/orc_empty_strings.q | 4 +- .../clientpositive/orc_ends_with_nulls.q | 2 +- .../test/queries/clientpositive/orc_min_max.q | 32 + .../queries/clientpositive/orc_ppd_char.q | 76 + .../queries/clientpositive/orc_ppd_date.q | 97 + .../queries/clientpositive/orc_ppd_decimal.q | 151 + .../queries/clientpositive/orc_ppd_varchar.q | 76 + .../clientpositive/orc_predicate_pushdown.q | 6 +- .../clientpositive/orc_split_elimination.q | 168 ++ .../clientpositive/orc_vectorization_ppd.q | 69 + .../clientpositive/order_within_subquery.q | 19 + .../queries/clientpositive/parallel_orderby.q | 4 +- .../queries/clientpositive/parquet_create.q | 36 + .../queries/clientpositive/parquet_ctas.q | 24 + .../clientpositive/parquet_partitioned.q | 34 + .../queries/clientpositive/parquet_types.q | 38 + .../test/queries/clientpositive/partcols1.q | 2 +- .../queries/clientpositive/partition_date.q | 51 +- .../queries/clientpositive/partition_date2.q | 6 +- .../clientpositive/partition_decode_name.q | 6 +- .../clientpositive/partition_special_char.q | 6 +- .../clientpositive/partition_type_check.q | 6 +- .../clientpositive/partition_varchar1.q | 8 +- .../clientpositive/partition_varchar2.q | 10 + .../partition_wise_fileformat17.q | 10 +- .../partition_wise_fileformat18.q | 19 + .../ql/src/test/queries/clientpositive/pcr.q | 2 +- .../test/queries/clientpositive/ppd_join4.q | 22 + .../queries/clientpositive/ppd_multi_insert.q | 12 +- .../queries/clientpositive/ppd_transform.q | 9 + .../test/queries/clientpositive/ppd_udtf.q | 12 + .../queries/clientpositive/ppd_union_view.q | 12 +- .../queries/clientpositive/ppr_pushdown.q | 18 +- .../queries/clientpositive/ppr_pushdown2.q | 16 +- .../test/queries/clientpositive/progress_1.q | 2 +- .../ql/src/test/queries/clientpositive/ptf.q | 2 +- .../test/queries/clientpositive/ptf_decimal.q | 4 +- .../clientpositive/ptf_general_queries.q | 2 +- .../queries/clientpositive/ptf_matchpath.q | 2 +- .../test/queries/clientpositive/ptf_rcfile.q | 2 +- .../clientpositive/ptf_register_tblfn.q | 2 +- .../test/queries/clientpositive/ptf_seqfile.q | 2 +- .../clientpositive/ql_rewrite_gbtoidx.q | 6 +- .../src/test/queries/clientpositive/quote2.q | 2 + .../queries/clientpositive/quotedid_alter.q | 21 + .../queries/clientpositive/quotedid_basic.q | 34 + .../clientpositive/quotedid_partition.q | 24 + .../queries/clientpositive/quotedid_skew.q | 26 + .../queries/clientpositive/quotedid_smb.q | 34 + .../clientpositive/quotedid_tblproperty.q | 8 + .../queries/clientpositive/rcfile_bigdata.q | 2 +- .../test/queries/clientpositive/regex_col.q | 2 + .../queries/clientpositive/remote_script.q | 2 +- .../src/test/queries/clientpositive/repair.q | 10 +- .../clientpositive/root_dir_external_table.q | 11 + .../queries/clientpositive/schemeAuthority2.q | 4 +- .../test/queries/clientpositive/scriptfile1.q | 4 +- .../queries/clientpositive/scriptfile1_win.q | 16 + .../clientpositive/select_dummy_source.q | 33 + .../test/queries/clientpositive/serde_regex.q | 10 +- .../clientpositive/set_processor_namespaces.q | 2 +- .../clientpositive/show_indexes_edge_cases.q | 3 +- .../clientpositive/show_indexes_syntax.q | 1 + .../queries/clientpositive/show_partitions.q | 3 +- .../test/queries/clientpositive/show_roles.q | 4 + .../queries/clientpositive/show_tablestatus.q | 1 + .../test/queries/clientpositive/skewjoin.q | 8 +- .../queries/clientpositive/skewjoin_noskew.q | 9 + .../clientpositive/skewjoin_union_remove_1.q | 4 +- .../clientpositive/skewjoin_union_remove_2.q | 6 +- .../queries/clientpositive/skewjoinopt1.q | 4 +- .../queries/clientpositive/skewjoinopt10.q | 2 +- .../queries/clientpositive/skewjoinopt11.q | 4 +- .../queries/clientpositive/skewjoinopt12.q | 4 +- .../queries/clientpositive/skewjoinopt13.q | 6 +- .../queries/clientpositive/skewjoinopt14.q | 6 +- .../queries/clientpositive/skewjoinopt15.q | 4 +- .../queries/clientpositive/skewjoinopt16.q | 4 +- .../queries/clientpositive/skewjoinopt17.q | 8 +- .../queries/clientpositive/skewjoinopt18.q | 4 +- .../queries/clientpositive/skewjoinopt19.q | 4 +- .../queries/clientpositive/skewjoinopt2.q | 4 +- .../queries/clientpositive/skewjoinopt20.q | 4 +- .../queries/clientpositive/skewjoinopt3.q | 4 +- .../queries/clientpositive/skewjoinopt4.q | 4 +- .../queries/clientpositive/skewjoinopt5.q | 4 +- .../queries/clientpositive/skewjoinopt6.q | 4 +- .../queries/clientpositive/skewjoinopt7.q | 6 +- .../queries/clientpositive/skewjoinopt8.q | 6 +- .../queries/clientpositive/skewjoinopt9.q | 4 +- .../queries/clientpositive/smb_mapjoin_1.q | 6 +- .../queries/clientpositive/smb_mapjoin_10.q | 8 +- .../queries/clientpositive/smb_mapjoin_2.q | 6 +- .../queries/clientpositive/smb_mapjoin_25.q | 6 +- .../queries/clientpositive/smb_mapjoin_3.q | 6 +- .../queries/clientpositive/smb_mapjoin_4.q | 6 +- .../queries/clientpositive/smb_mapjoin_5.q | 6 +- .../queries/clientpositive/smb_mapjoin_7.q | 4 +- .../queries/clientpositive/smb_mapjoin_8.q | 2 +- .../src/test/queries/clientpositive/source.q | 2 +- .../src/test/queries/clientpositive/split.q | 8 + .../src/test/queries/clientpositive/stats1.q | 2 +- .../src/test/queries/clientpositive/stats11.q | 18 +- .../src/test/queries/clientpositive/stats18.q | 2 +- .../src/test/queries/clientpositive/stats19.q | 4 +- .../src/test/queries/clientpositive/stats3.q | 4 +- .../src/test/queries/clientpositive/stats4.q | 4 +- .../clientpositive/stats_aggregator_error_1.q | 4 +- .../queries/clientpositive/stats_counter.q | 16 + .../stats_counter_partitioned.q | 45 + .../clientpositive/stats_invalidation.q | 15 + .../clientpositive/stats_list_bucket.q | 45 + .../queries/clientpositive/stats_noscan_2.q | 10 +- .../queries/clientpositive/stats_only_null.q | 41 + .../clientpositive/stats_publisher_error_1.q | 2 +- .../src/test/queries/clientpositive/statsfs.q | 63 + .../test/queries/clientpositive/str_to_map.q | 4 +- .../ql/src/test/queries/clientpositive/subq.q | 6 +- .../clientpositive/subq_where_serialization.q | 5 + .../queries/clientpositive/subquery_alias.q | 16 + .../queries/clientpositive/subquery_exists.q | 45 + .../clientpositive/subquery_exists_having.q | 60 + .../test/queries/clientpositive/subquery_in.q | 163 ++ .../clientpositive/subquery_in_having.q | 104 + .../clientpositive/subquery_multiinsert.q | 82 + .../clientpositive/subquery_notexists.q | 41 + .../subquery_notexists_having.q | 46 + .../queries/clientpositive/subquery_notin.q | 143 + .../clientpositive/subquery_notin_having.q | 74 + .../subquery_unqualcolumnrefs.q | 83 + .../queries/clientpositive/subquery_views.q | 48 + .../symlink_text_input_format.q | 9 +- .../clientpositive/table_access_keys_stats.q | 2 +- .../clientpositive/test_boolean_whereclause.q | 2 +- .../src/test/queries/clientpositive/tez_dml.q | 40 + .../test/queries/clientpositive/tez_fsstat.q | 19 + .../tez_insert_overwrite_local_directory_1.q | 5 + .../queries/clientpositive/tez_join_tests.q | 12 + .../clientpositive/tez_joins_explain.q | 5 + .../clientpositive/tez_schema_evolution.q | 14 + .../test/queries/clientpositive/tez_union.q | 94 + .../test/queries/clientpositive/timestamp_1.q | 16 +- .../test/queries/clientpositive/timestamp_2.q | 16 +- .../test/queries/clientpositive/timestamp_3.q | 8 +- .../clientpositive/timestamp_comparison.q | 3 +- .../queries/clientpositive/timestamp_lazy.q | 2 +- .../queries/clientpositive/timestamp_null.q | 2 +- .../queries/clientpositive/timestamp_udf.q | 8 +- .../test/queries/clientpositive/transform1.q | 2 +- .../queries/clientpositive/truncate_column.q | 8 +- .../clientpositive/truncate_column_merge.q | 4 +- .../queries/clientpositive/truncate_table.q | 10 +- .../test/queries/clientpositive/type_cast_1.q | 2 + .../clientpositive/type_conversions_1.q | 1 + .../queries/clientpositive/type_widening.q | 1 + .../queries/clientpositive/udaf_collect_set.q | 11 + .../clientpositive/udaf_context_ngrams.q | 2 +- .../test/queries/clientpositive/udaf_corr.q | 2 +- .../queries/clientpositive/udaf_covar_pop.q | 2 +- .../queries/clientpositive/udaf_covar_samp.q | 2 +- .../test/queries/clientpositive/udaf_ngrams.q | 2 +- .../queries/clientpositive/udaf_percentile.q | 1 + .../udaf_percentile_approx_20.q | 8 +- .../udaf_percentile_approx_23.q | 8 +- .../queries/clientpositive/udaf_sum_list.q | 6 + .../src/test/queries/clientpositive/udf_E.q | 12 +- .../src/test/queries/clientpositive/udf_PI.q | 12 +- .../src/test/queries/clientpositive/udf_abs.q | 10 +- .../test/queries/clientpositive/udf_acos.q | 10 +- .../test/queries/clientpositive/udf_array.q | 6 +- .../clientpositive/udf_array_contains.q | 6 +- .../test/queries/clientpositive/udf_ascii.q | 6 +- .../test/queries/clientpositive/udf_asin.q | 10 +- .../test/queries/clientpositive/udf_atan.q | 10 +- .../test/queries/clientpositive/udf_between.q | 2 + .../src/test/queries/clientpositive/udf_bin.q | 6 +- .../queries/clientpositive/udf_bitmap_and.q | 8 +- .../queries/clientpositive/udf_bitmap_empty.q | 6 +- .../queries/clientpositive/udf_bitmap_or.q | 8 +- .../test/queries/clientpositive/udf_case.q | 25 +- .../queries/clientpositive/udf_case_thrift.q | 6 +- .../queries/clientpositive/udf_coalesce.q | 6 +- .../clientpositive/udf_compare_java_string.q | 2 + .../test/queries/clientpositive/udf_concat.q | 10 +- .../queries/clientpositive/udf_concat_ws.q | 8 +- .../test/queries/clientpositive/udf_conv.q | 20 +- .../src/test/queries/clientpositive/udf_cos.q | 6 +- .../clientpositive/udf_current_database.q | 26 + .../test/queries/clientpositive/udf_degrees.q | 12 +- .../src/test/queries/clientpositive/udf_div.q | 4 +- .../test/queries/clientpositive/udf_divide.q | 4 +- .../src/test/queries/clientpositive/udf_elt.q | 6 +- .../test/queries/clientpositive/udf_equal.q | 6 +- .../test/queries/clientpositive/udf_explode.q | 22 +- .../test/queries/clientpositive/udf_field.q | 12 +- .../queries/clientpositive/udf_find_in_set.q | 28 +- .../clientpositive/udf_format_number.q | 16 +- .../clientpositive/udf_get_json_object.q | 8 +- .../queries/clientpositive/udf_greaterthan.q | 4 +- .../clientpositive/udf_greaterthanorequal.q | 4 +- .../test/queries/clientpositive/udf_hash.q | 6 +- .../src/test/queries/clientpositive/udf_hex.q | 8 +- .../test/queries/clientpositive/udf_hour.q | 2 + .../src/test/queries/clientpositive/udf_if.q | 10 +- .../src/test/queries/clientpositive/udf_in.q | 4 +- .../test/queries/clientpositive/udf_in_file.q | 12 +- .../test/queries/clientpositive/udf_index.q | 1 + .../test/queries/clientpositive/udf_inline.q | 2 + .../test/queries/clientpositive/udf_instr.q | 6 +- .../clientpositive/udf_isnull_isnotnull.q | 2 + .../queries/clientpositive/udf_java_method.q | 6 +- .../test/queries/clientpositive/udf_length.q | 4 +- .../queries/clientpositive/udf_lessthan.q | 4 +- .../clientpositive/udf_lessthanorequal.q | 4 +- .../test/queries/clientpositive/udf_like.q | 4 +- .../test/queries/clientpositive/udf_locate.q | 6 +- .../clientpositive/udf_logic_java_boolean.q | 2 + .../test/queries/clientpositive/udf_lpad.q | 6 +- .../src/test/queries/clientpositive/udf_map.q | 6 +- .../queries/clientpositive/udf_map_keys.q | 6 +- .../queries/clientpositive/udf_map_values.q | 6 +- .../queries/clientpositive/udf_named_struct.q | 6 +- .../queries/clientpositive/udf_negative.q | 14 +- .../queries/clientpositive/udf_notequal.q | 2 + .../test/queries/clientpositive/udf_notop.q | 4 +- .../src/test/queries/clientpositive/udf_nvl.q | 5 +- .../test/queries/clientpositive/udf_pmod.q | 20 +- .../test/queries/clientpositive/udf_printf.q | 16 +- .../test/queries/clientpositive/udf_radians.q | 16 +- .../test/queries/clientpositive/udf_reflect.q | 6 +- .../queries/clientpositive/udf_reflect2.q | 2 + .../test/queries/clientpositive/udf_regexp.q | 4 +- .../test/queries/clientpositive/udf_repeat.q | 6 +- .../test/queries/clientpositive/udf_reverse.q | 2 +- .../test/queries/clientpositive/udf_round.q | 14 +- .../test/queries/clientpositive/udf_round_2.q | 8 +- .../test/queries/clientpositive/udf_round_3.q | 12 +- .../test/queries/clientpositive/udf_rpad.q | 6 +- .../test/queries/clientpositive/udf_second.q | 2 + .../test/queries/clientpositive/udf_sign.q | 20 +- .../src/test/queries/clientpositive/udf_sin.q | 6 +- .../test/queries/clientpositive/udf_size.q | 2 + .../queries/clientpositive/udf_sort_array.q | 14 +- .../test/queries/clientpositive/udf_space.q | 8 +- .../test/queries/clientpositive/udf_split.q | 6 +- .../test/queries/clientpositive/udf_struct.q | 6 +- .../test/queries/clientpositive/udf_substr.q | 18 +- .../src/test/queries/clientpositive/udf_tan.q | 10 +- .../queries/clientpositive/udf_testlength.q | 8 +- .../queries/clientpositive/udf_testlength2.q | 8 +- .../queries/clientpositive/udf_to_boolean.q | 60 +- .../test/queries/clientpositive/udf_to_byte.q | 22 +- .../queries/clientpositive/udf_to_double.q | 22 +- .../queries/clientpositive/udf_to_float.q | 22 +- .../test/queries/clientpositive/udf_to_long.q | 22 +- .../queries/clientpositive/udf_to_short.q | 22 +- .../queries/clientpositive/udf_to_string.q | 24 +- .../clientpositive/udf_to_unix_timestamp.q | 4 +- .../queries/clientpositive/udf_translate.q | 16 +- .../test/queries/clientpositive/udf_unhex.q | 6 +- .../test/queries/clientpositive/udf_union.q | 6 +- .../clientpositive/udf_unix_timestamp.q | 4 +- .../test/queries/clientpositive/udf_using.q | 15 + .../queries/clientpositive/udf_weekofyear.q | 4 +- .../test/queries/clientpositive/udf_when.q | 23 +- .../test/queries/clientpositive/udf_xpath.q | 12 +- .../clientpositive/udf_xpath_boolean.q | 14 +- .../queries/clientpositive/udf_xpath_double.q | 18 +- .../queries/clientpositive/udf_xpath_float.q | 18 +- .../queries/clientpositive/udf_xpath_int.q | 18 +- .../queries/clientpositive/udf_xpath_long.q | 18 +- .../queries/clientpositive/udf_xpath_short.q | 18 +- .../queries/clientpositive/udf_xpath_string.q | 18 +- .../queries/clientpositive/udtf_explode.q | 2 + .../queries/clientpositive/udtf_json_tuple.q | 14 +- .../clientpositive/udtf_parse_url_tuple.q | 12 +- .../queries/clientpositive/udtf_posexplode.q | 15 + .../src/test/queries/clientpositive/union.q | 6 +- .../src/test/queries/clientpositive/union34.q | 2 +- .../test/queries/clientpositive/union_date.q | 4 +- .../test/queries/clientpositive/union_null.q | 3 + .../queries/clientpositive/union_remove_1.q | 2 +- .../queries/clientpositive/union_remove_10.q | 2 +- .../queries/clientpositive/union_remove_11.q | 2 +- .../queries/clientpositive/union_remove_12.q | 2 +- .../queries/clientpositive/union_remove_13.q | 2 +- .../queries/clientpositive/union_remove_14.q | 2 +- .../queries/clientpositive/union_remove_15.q | 2 +- .../queries/clientpositive/union_remove_16.q | 2 +- .../queries/clientpositive/union_remove_17.q | 2 +- .../queries/clientpositive/union_remove_18.q | 2 +- .../queries/clientpositive/union_remove_19.q | 2 +- .../queries/clientpositive/union_remove_2.q | 2 +- .../queries/clientpositive/union_remove_20.q | 2 +- .../queries/clientpositive/union_remove_21.q | 2 +- .../queries/clientpositive/union_remove_22.q | 2 +- .../queries/clientpositive/union_remove_23.q | 2 +- .../queries/clientpositive/union_remove_24.q | 2 +- .../queries/clientpositive/union_remove_3.q | 2 +- .../queries/clientpositive/union_remove_4.q | 2 +- .../queries/clientpositive/union_remove_5.q | 2 +- .../queries/clientpositive/union_remove_6.q | 2 +- .../queries/clientpositive/union_remove_7.q | 2 +- .../queries/clientpositive/union_remove_8.q | 2 +- .../queries/clientpositive/union_remove_9.q | 2 +- .../queries/clientpositive/union_top_level.q | 106 + .../test/queries/clientpositive/union_view.q | 1 + .../test/queries/clientpositive/uniquejoin.q | 6 +- .../test/queries/clientpositive/varchar_1.q | 2 +- .../queries/clientpositive/varchar_cast.q | 1 + .../clientpositive/varchar_comparison.q | 1 + .../queries/clientpositive/varchar_join1.q | 6 +- .../queries/clientpositive/varchar_serde.q | 2 +- .../queries/clientpositive/varchar_union1.q | 6 +- .../clientpositive/vector_between_in.q | 35 + .../queries/clientpositive/vector_coalesce.q | 32 + .../clientpositive/vector_decimal_aggregate.q | 20 + .../clientpositive/vector_decimal_cast.q | 5 + .../vector_decimal_expressions.q | 5 + .../clientpositive/vector_decimal_mapjoin.q | 19 + .../vector_decimal_math_funcs.q | 77 + .../clientpositive/vector_left_outer_join.q | 21 + .../vector_non_string_partition.q | 17 + .../queries/clientpositive/vectorization_0.q | 27 + .../queries/clientpositive/vectorization_1.q | 21 + .../queries/clientpositive/vectorization_10.q | 24 + .../queries/clientpositive/vectorization_11.q | 15 + .../queries/clientpositive/vectorization_12.q | 32 + .../queries/clientpositive/vectorization_13.q | 31 + .../queries/clientpositive/vectorization_14.q | 33 + .../queries/clientpositive/vectorization_15.q | 31 + .../queries/clientpositive/vectorization_16.q | 20 + .../queries/clientpositive/vectorization_2.q | 23 + .../queries/clientpositive/vectorization_3.q | 25 + .../queries/clientpositive/vectorization_4.q | 23 + .../queries/clientpositive/vectorization_5.q | 20 + .../queries/clientpositive/vectorization_6.q | 21 + .../queries/clientpositive/vectorization_7.q | 25 + .../queries/clientpositive/vectorization_8.q | 23 + .../queries/clientpositive/vectorization_9.q | 24 + .../vectorization_decimal_date.q | 4 + .../clientpositive/vectorization_div0.q | 24 + .../clientpositive/vectorization_limit.q | 37 + .../clientpositive/vectorization_nested_udf.q | 3 + .../clientpositive/vectorization_not.q | 27 + .../clientpositive/vectorization_part.q | 7 + .../vectorization_part_project.q | 7 + .../clientpositive/vectorization_pushdown.q | 4 + .../vectorization_short_regress.q | 852 ++++++ .../vectorized_bucketmapjoin1.q | 46 + .../queries/clientpositive/vectorized_case.q | 37 + .../queries/clientpositive/vectorized_casts.q | 149 + .../clientpositive/vectorized_context.q | 47 + .../clientpositive/vectorized_date_funcs.q | 122 + .../clientpositive/vectorized_distinct_gby.q | 12 + .../clientpositive/vectorized_mapjoin.q | 12 + .../clientpositive/vectorized_math_funcs.q | 107 + .../vectorized_nested_mapjoin.q | 8 + .../vectorized_rcfile_columnar.q | 18 + .../clientpositive/vectorized_shufflejoin.q | 10 + .../clientpositive/vectorized_string_funcs.q | 46 + .../vectorized_timestamp_funcs.q | 124 + .../ql/src/test/queries/clientpositive/view.q | 4 +- .../test/queries/clientpositive/view_cast.q | 6 +- .../test/queries/clientpositive/windowing.q | 2 +- .../windowing_adjust_rowcontainer_sz.q | 2 +- .../clientpositive/windowing_columnPruning.q | 2 +- .../clientpositive/windowing_expressions.q | 6 +- .../windowing_multipartitioning.q | 4 +- .../queries/clientpositive/windowing_navfn.q | 6 +- .../queries/clientpositive/windowing_ntile.q | 4 +- .../queries/clientpositive/windowing_rank.q | 4 +- .../queries/clientpositive/windowing_udaf.q | 2 +- .../queries/clientpositive/windowing_udaf2.q | 4 + .../clientpositive/windowing_windowspec.q | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 2 - .../sql/hive/api/java/JavaHiveQLSuite.scala | 6 +- .../hive/execution/HiveComparisonTest.scala | 19 +- .../sql/hive/execution/HiveQuerySuite.scala | 16 +- .../org/apache/spark/sql/hive/Shim.scala | 89 - .../org/apache/spark/sql/hive/Shim12.scala | 96 + .../org/apache/spark/sql/hive/Shim.scala | 170 -- .../org/apache/spark/sql/hive/Shim13.scala | 169 ++ 8166 files changed, 38307 insertions(+), 47487 deletions(-) create mode 100644 sql/hive/src/test/resources/data/files/ProxyAuth.res create mode 100644 sql/hive/src/test/resources/data/files/alltypes.txt create mode 100644 sql/hive/src/test/resources/data/files/alltypes2.txt create mode 100644 sql/hive/src/test/resources/data/files/alltypesorc create mode 100644 sql/hive/src/test/resources/data/files/char_varchar_udf.txt create mode 100644 sql/hive/src/test/resources/data/files/decimal.txt create mode 100644 sql/hive/src/test/resources/data/files/dept.txt create mode 100644 sql/hive/src/test/resources/data/files/emp.txt create mode 100644 sql/hive/src/test/resources/data/files/exported_table/_metadata create mode 100644 sql/hive/src/test/resources/data/files/exported_table/data/data create mode 100644 sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt create mode 100644 sql/hive/src/test/resources/data/files/futurama_episodes.avro create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt rename sql/hive/src/test/resources/{golden/alter3-1-75be487df30e301e156a22eee075633d => data/files/header_footer_table_3/empty1.txt} (100%) rename sql/hive/src/test/resources/{golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 => data/files/header_footer_table_3/empty2.txt} (100%) create mode 100644 sql/hive/src/test/resources/data/files/input.txt create mode 100644 sql/hive/src/test/resources/data/files/keystore.jks create mode 100644 sql/hive/src/test/resources/data/files/kv9.txt create mode 100644 sql/hive/src/test/resources/data/files/loc.txt create mode 100644 sql/hive/src/test/resources/data/files/non_ascii_tbl.txt create mode 100644 sql/hive/src/test/resources/data/files/orc_split_elim.orc create mode 100644 sql/hive/src/test/resources/data/files/parquet_create.txt create mode 100644 sql/hive/src/test/resources/data/files/parquet_partitioned.txt create mode 100644 sql/hive/src/test/resources/data/files/parquet_types.txt create mode 100644 sql/hive/src/test/resources/data/files/person+age.txt create mode 100644 sql/hive/src/test/resources/data/files/posexplode_data.txt create mode 100644 sql/hive/src/test/resources/data/files/sample.json create mode 100644 sql/hive/src/test/resources/data/files/truststore.jks create mode 100644 sql/hive/src/test/resources/data/scripts/input20_script.py delete mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 delete mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c delete mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d rename sql/hive/src/test/resources/golden/{alter5-1-b8349afaf8e62dc6608a889c04ee3d4b => alter3-1-47f70392b97b94cdb425b25bde204f58} (100%) rename sql/hive/src/test/resources/golden/{alter5-15-b8349afaf8e62dc6608a889c04ee3d4b => alter3-21-91e32b3028ecc352dad8884829148311} (100%) rename sql/hive/src/test/resources/golden/{alter_merge_2-3-cfef140167765d259320ed1e8aba718d => alter5-1-cbad54fbd4a08fc8717708f93358ec3e} (100%) rename sql/hive/src/test/resources/golden/{alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be => alter5-15-cbad54fbd4a08fc8717708f93358ec3e} (100%) rename sql/hive/src/test/resources/golden/{alter_merge_2-5-43bd090cda258e25037e7c32d500a85f => alter_merge_2-3-bc9bb363f9a2026cfc70a31bb4551352} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a => alter_merge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 => alter_merge_2-5-6319bf26f3739260b1a77e2ea89ef147} (100%) delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 rename sql/hive/src/test/resources/golden/{alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 => alter_partition_protect_mode-1-e11f1476d2f30ab7a83f95013809f9e6} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 => alter_partition_protect_mode-2-29b4b67965f24c40bcc426043efb892d} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e => alter_partition_protect_mode-3-2100fc7853625d8c7dad5c0ee70d4690} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 => alter_partition_protect_mode-4-c3fa911536c75e87febf2874e7889879} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f => alter_rename_partition-19-d862c00e5a37ae841a6c8ec6c5d68e8c} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 => alter_rename_partition-4-833254c8f7c5b1248e1a249560652627} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 => alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d => alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d} (100%) rename sql/hive/src/test/resources/golden/{combine3-0-84b74227c9f1563f530cd3ac3b333e54 => alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c delete mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c delete mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb delete mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 rename sql/hive/src/test/resources/golden/{combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c => ambiguous_col-0-e91e3e5a22029b9b979ccbbef97add66} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d => ambiguous_col-1-b4fe82679efdf6a15e9ecff53baf8d8d} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 => ambiguous_col-2-dadfa3854356dead14b93c5a71a5d8ab} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 => ambiguous_col-3-70509ccd2765d90b98666b6dff8afe1b} (100%) rename sql/hive/src/test/resources/golden/{combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 => annotate_stats_join-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c => annotate_stats_join-1-84e7846d50fc15e836c83911ce039871} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 => annotate_stats_join-10-a59dc1c01d48c82d46a5beab759f618d} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 => annotate_stats_join-11-3e0ade2476221f6396381f55a3b82e60} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 => annotate_stats_join-12-cef44682d339a67ba765f854da21f976} (100%) rename sql/hive/src/test/resources/golden/{alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c => annotate_stats_join-13-6292c27f0c4eaeab2a5e6c317e3afa2e} (100%) rename sql/hive/src/test/resources/golden/{alter_rename_partition-4-3479a886936571d5028971aecade705f => annotate_stats_join-14-4f3042b9feebd00c540ddac40e7254d1} (100%) rename sql/hive/src/test/resources/golden/{alter_varchar2-2-325238d61f56d84c17e29033105d7b19 => annotate_stats_join-15-a2f3b8a636e46e5df514df46c452855f} (100%) rename sql/hive/src/test/resources/golden/{alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 => annotate_stats_join-16-a75699a21ea36d962a1ba45bd5a12f26} (100%) rename sql/hive/src/test/resources/golden/{alter_view_rename-0-bb255b994b5207324fba6988caa937e6 => annotate_stats_join-17-64380f94a431e3a1698aa2edd3d0a6b2} (100%) rename sql/hive/src/test/resources/golden/{alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d => annotate_stats_join-18-222d1fcce7420950436e00d3a1bba957} (100%) rename sql/hive/src/test/resources/golden/{alter_view_rename-3-95655e33f22fc8f66549a9708812589a => annotate_stats_join-19-dea5f8f5c703583c7a3bdcb62cd3d589} (100%) rename sql/hive/src/test/resources/golden/{ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d => annotate_stats_join-2-40548ec2313af8dbdcbb8ad0477d8600} (100%) rename sql/hive/src/test/resources/golden/{ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab => annotate_stats_join-20-2d5e186b97166e3a625a169d0d73f9c8} (100%) rename sql/hive/src/test/resources/golden/{ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b => annotate_stats_join-21-ed73d4b01424287148347ccf721b37e2} (100%) rename sql/hive/src/test/resources/golden/{archive-10-f845008104fd12eb0f13f4b113f95cf1 => annotate_stats_join-22-2cf93da6bb0efdafeaa989c9df236701} (100%) rename sql/hive/src/test/resources/golden/{archive-12-f07653bca86e1ecb614ffd0296790d05 => annotate_stats_join-23-fa90806f6137300311df032e28df3d4c} (100%) rename sql/hive/src/test/resources/golden/{archive-16-892147913578bcf60620b7dd73893dd0 => annotate_stats_join-24-4a6976344eeae35e059285ed78f9feb3} (100%) rename sql/hive/src/test/resources/golden/{archive-18-8f980275ab3a5bcfc1784f4acd46447a => annotate_stats_join-3-26f82fb6734d5bc6f7159b06c0949178} (100%) rename sql/hive/src/test/resources/golden/{archive-19-f8a52a8a40141409a667a9ba2cf9630f => annotate_stats_join-4-a598c93d86a646cfa6ea86da54bce9b8} (100%) rename sql/hive/src/test/resources/golden/{archive-2-713efc113418b01f76ffd589840193c8 => annotate_stats_join-5-d12ba848d20d1def95379c490068f050} (100%) rename sql/hive/src/test/resources/golden/{archive-21-f07653bca86e1ecb614ffd0296790d05 => annotate_stats_join-6-96a4806e61c5d882affac57e115f285f} (100%) rename sql/hive/src/test/resources/golden/{archive-23-892147913578bcf60620b7dd73893dd0 => annotate_stats_join-7-38be41d6328f57350a78fb2097d1ebd2} (100%) rename sql/hive/src/test/resources/golden/{archive-25-56bea24ffa83d9b5932893a8ff1cb44f => annotate_stats_join-8-eb11e867a36e2114ef0f0f001e01984c} (100%) rename sql/hive/src/test/resources/golden/{archive-26-44fa61c2bd0fd9acfa0d889e59880d8a => annotate_stats_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd} (100%) rename sql/hive/src/test/resources/golden/{combine3-15-7cccbdffc32975f8935eeba14a28147 => annotate_stats_limit-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{archive-27-9ae773ebe64a3d437a035e9d94f49e5 => annotate_stats_limit-1-26f82fb6734d5bc6f7159b06c0949178} (100%) rename sql/hive/src/test/resources/golden/{archive-29-a0c6922e3a1dca51861c8a872dc8af19 => annotate_stats_limit-10-a89c94fd0de0cfc96725fea890987cc0} (100%) rename sql/hive/src/test/resources/golden/{archive-3-27ad2962fed131f51ba802596ba37278 => annotate_stats_limit-2-eb11e867a36e2114ef0f0f001e01984c} (100%) rename sql/hive/src/test/resources/golden/{archive-31-cd46bc635e3010cf1b990a652a584a09 => annotate_stats_limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd} (100%) rename sql/hive/src/test/resources/golden/{archive-32-27ad2962fed131f51ba802596ba37278 => annotate_stats_limit-4-4252006172a476fbc591eebee49bffa3} (100%) rename sql/hive/src/test/resources/golden/{archive-4-3e95421993ab28d18245ec2340f580a3 => annotate_stats_limit-5-a2f3b8a636e46e5df514df46c452855f} (100%) rename sql/hive/src/test/resources/golden/{archive-5-c0c18ac884677231a41eea8d980d0451 => annotate_stats_limit-6-f1fd6c403a9787947877f48c772afe96} (100%) rename sql/hive/src/test/resources/golden/{archive-6-528ab9750a558af7f1a43b3108e793dd => annotate_stats_limit-7-69b422026fce4e2d2cde9a478d6aaa40} (100%) rename sql/hive/src/test/resources/golden/{archive-7-e8d1d10c308a73eef78dde414a5e40ca => annotate_stats_limit-8-72f5e55c1e244084eea255c32a6f58eb} (100%) rename sql/hive/src/test/resources/golden/{archive-8-af459a0264559a2aeaa1341ce779ab3c => annotate_stats_limit-9-9da67c62d5e3145d450ad7953b33361f} (100%) rename sql/hive/src/test/resources/golden/{combine3-2-c95dc367df88c9e5cf77157f29ba2daf => annotate_stats_part-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 => annotate_stats_part-1-3c29684bfd2df7439ee0551eb42cfa0} (100%) rename sql/hive/src/test/resources/golden/{archive-9-48b10f27e1459bb8e62d6c71484e2cf => annotate_stats_part-10-5ba0295bfe42deb678e59b3a330b14ff} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 => annotate_stats_part-11-dbdbe2e04c5dad6c8af78b6386b329} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 => annotate_stats_part-12-2856fb4f4af8932804cb238429d9de6f} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 => annotate_stats_part-13-4fa8b0f9fb0202ac0e82fb87538d6445} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a => annotate_stats_part-14-62c557bfb7bf70694a32ebceea329ebd} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f => annotate_stats_part-15-f796cd035603726a5c4ce3e71194822b} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 => annotate_stats_part-16-45eb5544124364714549f199f9c2b2ac} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 => annotate_stats_part-17-ad61ebd912b6bef3778c4ff38c0be5ab} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 => annotate_stats_part-18-b9f2dff7c7b57412cea44433ea810fa7} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f => annotate_stats_part-19-10cab43c2966718bb39e2f22365cd6c1} (100%) rename sql/hive/src/test/resources/golden/{combine3-4-84967075baa3e56fff2a23f8ab9ba076 => annotate_stats_part-2-a4fb8359a2179ec70777aad6366071b7} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a => annotate_stats_part-20-51ec5046b50543899ed54c9fc2b885af} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 => annotate_stats_part-21-d69cb7b7770b51bc4b99d5d0f74d4662} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 => annotate_stats_part-22-22f430881fbfe92736988c2a4184fe0c} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 => annotate_stats_part-23-2719ee3f0b8b07f1e5c80f8329b9f87f} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 => annotate_stats_part-24-1f7bdd816b83f45a6d53d08b6866109f} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 => annotate_stats_part-25-d1599e385e8bad6cd62d564a6fd0536f} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 => annotate_stats_part-26-ec26bcb2587f4638b364ba494ac373e0} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 => annotate_stats_part-27-7804e2e230a42a46d72546fa2381b423} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd => annotate_stats_part-28-40f2a1f06d181ef93edf42a591cbf15e} (100%) rename sql/hive/src/test/resources/golden/{combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea => annotate_stats_part-3-16367c381d4b189b3640c92511244bfe} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca => annotate_stats_part-4-397e834a25a20b68aed5e87d55ea6174} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c => annotate_stats_part-5-cef44682d339a67ba765f854da21f976} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf => annotate_stats_part-6-f4263aca1612c05cb08242448191ad05} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 => annotate_stats_part-7-b222533ddda09918c4f5acc408bf1a02} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-12-8419ad4ed6683ebd15f993f703975b31 => annotate_stats_part-8-84e3979946de67779a9704a3adc2184f} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d => annotate_stats_part-9-c6c38e93babafe56cd4f177a17d37b8} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b => annotate_stats_table-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f => annotate_stats_table-1-3c29684bfd2df7439ee0551eb42cfa0} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-2-cd46bc635e3010cf1b990a652a584a09 => annotate_stats_table-10-a7419af512a6c0b60ec51e556608cdad} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-3-27ad2962fed131f51ba802596ba37278 => annotate_stats_table-11-4be11140c2e21b3a172224d103844785} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-4-3e95421993ab28d18245ec2340f580a3 => annotate_stats_table-12-84bc7cf67544461e9102820ed410dfb6} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-5-c0c18ac884677231a41eea8d980d0451 => annotate_stats_table-13-d2acfdf279aad035b31ed61e87bff6} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a => annotate_stats_table-14-60d1c44b8412c4983265e320ecfb25f0} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca => annotate_stats_table-15-7b185296b6a0c6ceff2df1f89fecc6b4} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-8-af459a0264559a2aeaa1341ce779ab3c => annotate_stats_table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf => annotate_stats_table-17-cd84d629d569a09d139f38dae01e677c} (100%) rename sql/hive/src/test/resources/golden/{auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf => annotate_stats_table-18-7012af4092c18075f4f3a00b80f491ee} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 => annotate_stats_table-2-84e7846d50fc15e836c83911ce039871} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-13-63241e3791725baad8baa00fb833ef5e => annotate_stats_table-3-a598c93d86a646cfa6ea86da54bce9b8} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f => annotate_stats_table-4-d12ba848d20d1def95379c490068f050} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-15-187fd938996ae7d96e60475fb69b8d35 => annotate_stats_table-5-4a8ad142e57c5dce2623e92f5e5ad15a} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-20-e67740fb52998f1d3afcfa667505cf7 => annotate_stats_table-6-a59dc1c01d48c82d46a5beab759f618d} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b => annotate_stats_table-7-6292c27f0c4eaeab2a5e6c317e3afa2e} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-22-a1d339a0d904c3f35771192a026c7f9c => annotate_stats_table-8-43cb040e2fe01904bc52e198fcae9b3d} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-23-1948951cc3c06cdf962d59e932a84588 => annotate_stats_table-9-eadfdc61d22bd22bbf5a69370908a82e} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 => annotate_stats_union-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 => annotate_stats_union-1-26f82fb6734d5bc6f7159b06c0949178} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b => annotate_stats_union-10-3aa3eb68d092ea0157c5426a5f2ca3f9} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-32-3c52df82c7d78501610f3f898103f753 => annotate_stats_union-11-c1302094d7b4456550826535b529468b} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c => annotate_stats_union-12-3af760f960a2e0995d1bc4afef0c5aa0} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 => annotate_stats_union-13-26f82fb6734d5bc6f7159b06c0949178} (100%) rename sql/hive/src/test/resources/golden/{auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 => annotate_stats_union-14-eb11e867a36e2114ef0f0f001e01984c} (100%) rename sql/hive/src/test/resources/golden/{auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 => annotate_stats_union-15-e0b7cb2226234fbea8ae11ea7d9d84bd} (100%) rename sql/hive/src/test/resources/golden/{auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 => annotate_stats_union-16-4252006172a476fbc591eebee49bffa3} (100%) rename sql/hive/src/test/resources/golden/{auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b => annotate_stats_union-17-a2f3b8a636e46e5df514df46c452855f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 => annotate_stats_union-18-58d55f8c8b2489726232a00254499144} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 => annotate_stats_union-19-70c851c23a7ac89f0366b9c26085cc3e} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 => annotate_stats_union-2-eb11e867a36e2114ef0f0f001e01984c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 => annotate_stats_union-20-222d1fcce7420950436e00d3a1bba957} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a => annotate_stats_union-21-a5503df8f50e057415719bcfe4c69f13} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab => annotate_stats_union-22-a23089e5a76dc35df04d9936d60c4565} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 => annotate_stats_union-3-e0b7cb2226234fbea8ae11ea7d9d84bd} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 => annotate_stats_union-4-4252006172a476fbc591eebee49bffa3} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d => annotate_stats_union-5-a2f3b8a636e46e5df514df46c452855f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 => annotate_stats_union-6-222d1fcce7420950436e00d3a1bba957} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 => annotate_stats_union-7-aab1a7b7e1fe4d061a580126d67dfd0a} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 => annotate_stats_union-8-172e0d089bd5bcbaf54775a618d826bb} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 => annotate_stats_union-9-69b422026fce4e2d2cde9a478d6aaa40} (100%) delete mode 100644 sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 delete mode 100644 sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 delete mode 100644 sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 delete mode 100644 sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 delete mode 100644 sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a delete mode 100644 sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_join32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a => auto_join32-13-ee2dcaae78ae900ffce8d19fbadc3735} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab => auto_join32-14-7927c2ce644d1ce1de251405c8563e99} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 => auto_join32-15-4cd3b51861720ac06c6deb818c83670} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 => auto_join32-20-693874ea8e06a8b155394ac27af2b1a7} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d => auto_join32-21-bf8a1bb0baaae9fbf1c3aa656f991f42} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_join32-22-3d14d63e996851f51a98f987995d8da6} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 => auto_join32-23-8b183ec2c164b3b530e802ffc880a5fa} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 => auto_join_filters-2-bee6095f42de6a16708c2f9addc1b9bd} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 => auto_join_filters-31-268d8fb3cb9b04eb269fe7ec40a24dfe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a => auto_join_filters-32-6dc6866a65c74d69538b776b41b06c16} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 => auto_join_filters-33-e884480a0f7273d3e2f2de2ba46b855c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 => auto_join_filters-34-98fd86aea9cacaa82d43c7468109dd33} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_join_nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a => auto_join_reordering_values-1-2bfb628930d072124636d21d82e3b462} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab => auto_join_reordering_values-4-11af6838bb9e04152c2f9a7e2044abe0} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 => auto_join_reordering_values-7-99fcaa5203ed3debb52c9086028dc8c2} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 => auto_sortmerge_join_1-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d => auto_sortmerge_join_1-10-5cba470fbd02e730781a3b63fd9aa3e2} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_sortmerge_join_1-11-337e909605c780d00ad8895686defa06} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_1-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_1-4-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a => auto_sortmerge_join_1-5-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 => auto_sortmerge_join_1-6-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_1-7-ae582a255a02d91674aab500aee79e20} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_1-8-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d => auto_sortmerge_join_1-9-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_sortmerge_join_11-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_11-10-5cba470fbd02e730781a3b63fd9aa3e2} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_11-11-337e909605c780d00ad8895686defa06} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 => auto_sortmerge_join_11-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f => auto_sortmerge_join_11-4-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_11-5-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_11-6-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 => auto_sortmerge_join_11-7-ae582a255a02d91674aab500aee79e20} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 => auto_sortmerge_join_11-8-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_11-9-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_12-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_12-10-5cba470fbd02e730781a3b63fd9aa3e2} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_12-11-337e909605c780d00ad8895686defa06} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a => auto_sortmerge_join_12-17-97ba394ab6aad2547f399ebbf757a4b6} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 => auto_sortmerge_join_12-18-73ee2d7b62e2aede20ca5de577cd7b7f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 => auto_sortmerge_join_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f => auto_sortmerge_join_12-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 => auto_sortmerge_join_12-4-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 => auto_sortmerge_join_12-5-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 => auto_sortmerge_join_12-6-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 => auto_sortmerge_join_12-7-ae582a255a02d91674aab500aee79e20} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c => auto_sortmerge_join_12-8-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 => auto_sortmerge_join_12-9-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 => auto_sortmerge_join_16-0-24ca942f094b14b92086305cc125e833} (100%) rename sql/hive/src/test/resources/golden/{decimal_precision-14-34916eb904b8113a401ce78e6941a204 => auto_sortmerge_join_16-1-16367c381d4b189b3640c92511244bfe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 => auto_sortmerge_join_16-10-4706d21b17f993cc1cc94be6b7e04c28} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a => auto_sortmerge_join_16-11-5e81d0b41cc58d8cc151046c7a111411} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_16-12-e8a77ff790bfd6740489fc4374ec6c3d} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_16-13-920818d557d2525dabb5c539a80a1bbb} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_16-14-455dfeeba27ecf53923db0cbf0aab908} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d => auto_sortmerge_join_16-15-11f98f575685beedc14a88fc47a61041} (100%) create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b => auto_sortmerge_join_16-2-7b4ad215fc2e75c71c6614a2b6322e8e} (100%) rename sql/hive/src/test/resources/golden/{escape1-0-a4fb8359a2179ec70777aad6366071b7 => auto_sortmerge_join_16-3-365488a703b0640acda73a7d7e6efa06} (100%) rename sql/hive/src/test/resources/golden/{escape1-1-683124e29877d2c5a96b95c8ddba97b7 => auto_sortmerge_join_16-4-d0ec6d66ff349db09fd455eec149efdb} (100%) rename sql/hive/src/test/resources/golden/{escape2-0-a4fb8359a2179ec70777aad6366071b7 => auto_sortmerge_join_16-5-3b0f76816be2c1b18a2058027a19bc9f} (100%) rename sql/hive/src/test/resources/golden/{escape2-1-683124e29877d2c5a96b95c8ddba97b7 => auto_sortmerge_join_16-6-86473a0498e4361e4db0b4a22f2e8571} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_sortmerge_join_16-7-7e87a31677022b6a1c360922ef74754e} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a => auto_sortmerge_join_16-9-ae5880516ea2f924cfbaeb919adc86e} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 => auto_sortmerge_join_2-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 => auto_sortmerge_join_2-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f => auto_sortmerge_join_2-3-aa6ec7f17b48cf49f02ab7367453ab39} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 => auto_sortmerge_join_2-4-66b07c93d79ed9958b8427dad16c3ef3} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 => auto_sortmerge_join_2-6-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_2-7-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d => auto_sortmerge_join_2-8-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_sortmerge_join_2-9-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 => auto_sortmerge_join_3-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 => auto_sortmerge_join_3-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 => auto_sortmerge_join_3-4-45b63361c1e5178b69a1531b238c8460} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f => auto_sortmerge_join_3-6-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_3-7-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_3-8-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 => auto_sortmerge_join_3-9-ae582a255a02d91674aab500aee79e20} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 => auto_sortmerge_join_4-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_4-10-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 => auto_sortmerge_join_4-11-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_4-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a => auto_sortmerge_join_4-3-aa6ec7f17b48cf49f02ab7367453ab39} (100%) rename sql/hive/src/test/resources/golden/{ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 => auto_sortmerge_join_4-4-66b07c93d79ed9958b8427dad16c3ef3} (100%) rename sql/hive/src/test/resources/golden/{ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1} (100%) rename sql/hive/src/test/resources/golden/{ba_table3-1-28b12606c5369c783e63c17826a18d0d => auto_sortmerge_join_4-6-45b63361c1e5178b69a1531b238c8460} (100%) rename sql/hive/src/test/resources/golden/{ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_4-7-ecca12a2f377c18c53563a534e9dd5a5} (100%) rename sql/hive/src/test/resources/golden/{ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 => auto_sortmerge_join_4-8-65930e1b01da720cf296ca3df668b58d} (100%) rename sql/hive/src/test/resources/golden/{binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 => auto_sortmerge_join_5-1-fdbb11a2de2777dfdd916b59764d5c8e} (100%) rename sql/hive/src/test/resources/golden/{binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef => auto_sortmerge_join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed} (100%) rename sql/hive/src/test/resources/golden/{binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 => auto_sortmerge_join_5-3-9878d6ab0fe143611c88ffc0602197e3} (100%) rename sql/hive/src/test/resources/golden/{binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_5-4-a576657b9fd1c23aed85a7409887b2fb} (100%) rename sql/hive/src/test/resources/golden/{binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 => auto_sortmerge_join_5-6-350b202868590b5edaed18007fd5cbbe} (100%) rename sql/hive/src/test/resources/golden/{binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e => auto_sortmerge_join_5-7-1155788c7c133a73c3609e8052accfa5} (100%) rename sql/hive/src/test/resources/golden/{binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_7-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 => auto_sortmerge_join_7-10-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 => auto_sortmerge_join_7-11-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d => auto_sortmerge_join_7-12-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c => auto_sortmerge_join_7-13-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 => auto_sortmerge_join_7-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c => auto_sortmerge_join_7-3-aa6ec7f17b48cf49f02ab7367453ab39} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 => auto_sortmerge_join_7-4-66b07c93d79ed9958b8427dad16c3ef3} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 => auto_sortmerge_join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd => auto_sortmerge_join_7-6-45b63361c1e5178b69a1531b238c8460} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 => auto_sortmerge_join_7-7-ecca12a2f377c18c53563a534e9dd5a5} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe => auto_sortmerge_join_7-8-65930e1b01da720cf296ca3df668b58d} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 => auto_sortmerge_join_8-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c => auto_sortmerge_join_8-10-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 => auto_sortmerge_join_8-11-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c => auto_sortmerge_join_8-12-5cba470fbd02e730781a3b63fd9aa3e2} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 => auto_sortmerge_join_8-13-337e909605c780d00ad8895686defa06} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f => auto_sortmerge_join_8-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 => auto_sortmerge_join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 => auto_sortmerge_join_8-4-45b63361c1e5178b69a1531b238c8460} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 => auto_sortmerge_join_8-6-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 => auto_sortmerge_join_8-7-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 => auto_sortmerge_join_8-8-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 => auto_sortmerge_join_8-9-ae582a255a02d91674aab500aee79e20} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 delete mode 100644 sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 delete mode 100644 sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 delete mode 100644 sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 rename sql/hive/src/test/resources/golden/{escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c => binary_constant-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 create mode 100644 sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 delete mode 100644 sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 delete mode 100644 sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 delete mode 100644 sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 rename sql/hive/src/test/resources/golden/{combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 => binarysortable_1-1-aca7ae366662c9698b5d2c01a6cb3948} (100%) delete mode 100644 sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 rename sql/hive/src/test/resources/golden/{combine2_win-12-5ac3e540fd24f94fee378e49597817b3 => char_varchar_udf-0-4de1b2fa0908a3d856474aae2bc38c08} (100%) rename sql/hive/src/test/resources/golden/{combine2_win-14-dd652175dac4463fed3c56aded11e6c1 => char_varchar_udf-1-5b1e7c580ed5e756d30a4c557af8902} (100%) rename sql/hive/src/test/resources/golden/{combine2_win-16-8e4598e3f0701478ed12042438699ce5 => char_varchar_udf-2-880ce74a83bb2bfb9c9bd584b8874ac} (100%) delete mode 100644 sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f delete mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce delete mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 delete mode 100644 sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 delete mode 100644 sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b delete mode 100644 sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc delete mode 100644 sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 rename sql/hive/src/test/resources/golden/{combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af => compute_stats_binary-1-44b15c6c9600109e064a5ea5f9c81051} (100%) rename sql/hive/src/test/resources/golden/{combine3-10-fb173ed4483e732d367e51f88be793b1 => compute_stats_boolean-1-72ee4bdd5cea69136940dc40e6890e1d} (100%) rename sql/hive/src/test/resources/golden/{combine3-16-6635f7f5c55557b06ad3acc321eaa739 => compute_stats_double-1-8f634b9e334fd58e71844e6283d9794d} (100%) rename sql/hive/src/test/resources/golden/{combine3-17-8cb751103da7c909276db6bddb50ae6a => compute_stats_long-1-9313f166464633b3929707d7ef11d758} (100%) rename sql/hive/src/test/resources/golden/{combine3-18-31fae7c6be75b97d475512bd75a58a0b => compute_stats_string-1-3491ef2747a8f34899108d4ae8ebc7eb} (100%) rename sql/hive/src/test/resources/golden/{combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f => correlationoptimizer1-26-5522db58d123d1bec48b6e71c1b258f3} (100%) rename sql/hive/src/test/resources/golden/{combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd => correlationoptimizer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65} (100%) create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f rename sql/hive/src/test/resources/golden/{combine3-22-11025483569617a9f014b5defd71e933 => correlationoptimizer1-35-e1d4857548e626bb31d70c096b8d0a95} (100%) create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f rename sql/hive/src/test/resources/golden/{combine3-23-4725c48df09565618cbffd05953a5f62 => correlationoptimizer1-38-ef6502d6b282c8a6d228bba395b24724} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 => correlationoptimizer1-39-ea87e76dba02a46cb958148333e397b7} (100%) rename sql/hive/src/test/resources/golden/{combine3-6-4725c48df09565618cbffd05953a5f62 => correlationoptimizer1-41-b79b220859c09354e23b533c105ccbab} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 => correlationoptimizer1-42-ea87e76dba02a46cb958148333e397b7} (100%) rename sql/hive/src/test/resources/golden/{combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 => correlationoptimizer1-44-638e5300f4c892c2bf27bd91a8f81b64} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad => correlationoptimizer1-45-66010469a9cdb66851da9a727ef9fdad} (100%) rename sql/hive/src/test/resources/golden/{combine3-8-68399bc39f71ddc99ed09ed9d2fd897b => correlationoptimizer1-47-3514c74c7f68f2d70cc6d51ac46c20} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad => correlationoptimizer1-48-66010469a9cdb66851da9a727ef9fdad} (100%) rename sql/hive/src/test/resources/golden/{escape2-3-1774adb1085f4ee6782a8dac0735399 => correlationoptimizer1-49-b9d963d24994c47c3776dda6f7d3881f} (100%) rename sql/hive/src/test/resources/golden/{combine3-9-b5703b76bbe99c41cbb63582a09a6e69 => correlationoptimizer1-50-7490df6719cd7e47aa08dbcbc3266a92} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba => correlationoptimizer1-51-e71195e7d9f557e2abc7f03462d22dba} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee => correlationoptimizer1-52-777edd9d575f3480ca6cebe4be57b1f6} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 => correlationoptimizer1-53-73da9fe2b0c2ee26c021ec3f2fa27272} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba => correlationoptimizer1-54-e71195e7d9f557e2abc7f03462d22dba} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a => correlationoptimizer1-55-b1e2ade89ae898650f0be4f796d8947b} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 => correlationoptimizer1-57-fcf9bcb522f542637ccdea863b408448} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-52-3070366869308907e54797927805603 => correlationoptimizer1-58-3070366869308907e54797927805603} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 => correlationoptimizer1-60-dad56e1f06c808b29e5dc8fb0c49efb2} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-55-3070366869308907e54797927805603 => correlationoptimizer1-61-3070366869308907e54797927805603} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 => correlationoptimizer1-62-b9d963d24994c47c3776dda6f7d3881f} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_long-1-a7bc730f9862198709539e35c0208248 => correlationoptimizer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 => correlationoptimizer1-64-a6bba6d9b422adb386b35c62cecb548} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee => correlationoptimizer1-65-777edd9d575f3480ca6cebe4be57b1f6} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f => correlationoptimizer1-66-d6bbaf0d40010159095e4cac025c50c5} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 => correlationoptimizer1-67-a6bba6d9b422adb386b35c62cecb548} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 => correlationoptimizer4-1-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 => correlationoptimizer4-3-ade68a23d7b1a4f328623bb5a0f07488} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab => correlationoptimizer4-5-b5777cff7c522c4b527f77988e7f6bf1} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 => count-1-c47c4abedf055b4e734232fd2e274d55} (100%) delete mode 100644 sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f delete mode 100644 sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d delete mode 100644 sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba delete mode 100644 sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 delete mode 100644 sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a delete mode 100644 sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 delete mode 100644 sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 delete mode 100644 sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 delete mode 100644 sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 delete mode 100644 sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a delete mode 100644 sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 delete mode 100644 sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 delete mode 100644 sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 delete mode 100644 sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 delete mode 100644 sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 delete mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 delete mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 delete mode 100644 sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af delete mode 100644 sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 delete mode 100644 sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 delete mode 100644 sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 delete mode 100644 sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 rename sql/hive/src/test/resources/golden/{correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 => create_nested_type-3-280fd0d9876e475d7dcf5383876ebc79} (100%) delete mode 100644 sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 rename sql/hive/src/test/resources/golden/{correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 => create_struct_table-1-719851d0e8b89b51bdc6be4177455a92} (100%) delete mode 100644 sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e delete mode 100644 sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b delete mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 delete mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc rename sql/hive/src/test/resources/golden/{correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 => cross_product_check_1-0-d782db598869f9b19e0fcf5ea2a83594} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 => cross_product_check_1-1-bd8395c55fa2fc80b68eb043b7020b76} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a => cross_product_check_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 => cross_product_check_1-3-32a3e6eb858d37f58e225d07fb323254} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 => cross_product_check_1-4-36a6b6fb71ea08ff817dd40d1ffdb970} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 => cross_product_check_1-5-103739f072b849d212dbc40919f92b74} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b => cross_product_check_1-6-9a202e3bf15be2a310504a50920e7d25} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 => cross_product_check_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 => cross_product_check_2-0-d782db598869f9b19e0fcf5ea2a83594} (100%) rename sql/hive/src/test/resources/golden/{count-1-3531872d964bc2b4f07d51b9d1629df2 => cross_product_check_2-1-bd8395c55fa2fc80b68eb043b7020b76} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 => cross_product_check_2-2-24ca942f094b14b92086305cc125e833} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee => cross_product_check_2-3-44d382ce6848d3f0b900b0808747d8e9} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a => cross_product_check_2-4-c14792ccac2ca64e3e9e21af4fd12d2c} (100%) rename sql/hive/src/test/resources/golden/{create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 => cross_product_check_2-5-32a3e6eb858d37f58e225d07fb323254} (100%) rename sql/hive/src/test/resources/golden/{create_1-1-c2351f011b8ea41ff7dfa8f195148da3 => cross_product_check_2-6-36a6b6fb71ea08ff817dd40d1ffdb970} (100%) rename sql/hive/src/test/resources/golden/{create_1-11-7daaeabd9c286e511e0628a32dc714d5 => cross_product_check_2-7-103739f072b849d212dbc40919f92b74} (100%) rename sql/hive/src/test/resources/golden/{create_1-14-437bd1acbae61f48e851addb769d3379 => cross_product_check_2-8-9a202e3bf15be2a310504a50920e7d25} (100%) rename sql/hive/src/test/resources/golden/{create_1-4-610b82bf7b0080d293977927e5ef780c => cross_product_check_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec delete mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 rename sql/hive/src/test/resources/golden/{create_1-5-c77b018276b1558c1d9462e0625e152e => database_drop-10-8db536f925bf0f5058f97897e145a661} (100%) rename sql/hive/src/test/resources/golden/{create_1-8-19331fe6a2a35f1171babfe4e1c86f59 => database_drop-11-1b0a6cff3151cfa0ef0a6f78587973a5} (100%) rename sql/hive/src/test/resources/golden/{database_drop-11-2ea883422b74b701711e14e61472ba06 => database_drop-12-2ea883422b74b701711e14e61472ba06} (100%) rename sql/hive/src/test/resources/golden/{database_drop-12-e02a53f7e798d2741152526516f14941 => database_drop-13-e02a53f7e798d2741152526516f14941} (100%) rename sql/hive/src/test/resources/golden/{create_big_view-0-bdf3c2e20793ef833f336a40791091d => database_drop-14-8f0fe60664d020b2a42c7f5c0c7bed35} (100%) rename sql/hive/src/test/resources/golden/{create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee => database_drop-15-7928ac876f76c990fa21d74b6c9e14f6} (100%) rename sql/hive/src/test/resources/golden/{create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e => database_drop-16-f9036ff3f0a3101373bdbc9a52faf00e} (100%) rename sql/hive/src/test/resources/golden/{create_escape-3-9541399cde42210bd7ac1beb07ceb14 => database_drop-17-95b906fd73935da8746b5277170e91e8} (100%) rename sql/hive/src/test/resources/golden/{create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b => database_drop-18-f65bf675b83871af7906741a60fa1318} (100%) rename sql/hive/src/test/resources/golden/{create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 => database_drop-19-d7fefbf585dbb67491e871ef58dca752} (100%) rename sql/hive/src/test/resources/golden/{create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 => database_drop-20-cbb84f0db4d55930a85cff28f7400bd0} (100%) rename sql/hive/src/test/resources/golden/{create_like-0-c2351f011b8ea41ff7dfa8f195148da3 => database_drop-21-eea49f066631be60519ae9d6b614d7d0} (100%) rename sql/hive/src/test/resources/golden/{create_like-2-a20451f152e68606cc5e373fe5fd86a => database_drop-22-85833c3a68c29098827e438ff580bb94} (100%) rename sql/hive/src/test/resources/golden/{create_like-4-39ead53334938635b60a5ffdaa2c9f86 => database_drop-23-84a5672989118a1b5792474c1469de90} (100%) rename sql/hive/src/test/resources/golden/{create_like-5-dc9de26002604e9e436135bd4b40636d => database_drop-24-441116797e8d95554c74472fa7644440} (100%) rename sql/hive/src/test/resources/golden/{database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 => database_drop-25-847ca97dd211889d83e5d487bfc44e23} (100%) rename sql/hive/src/test/resources/golden/{database_drop-4-d419f4ff197d4291208c2028cd158909 => database_drop-26-bea20178515df24fcca04c0384f1c1b7} (100%) rename sql/hive/src/test/resources/golden/{create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 => database_drop-27-cb4b90a7f63c1646319ee7bb014a7750} (100%) rename sql/hive/src/test/resources/golden/{create_like-8-639a13da6855b974fc2e170fd49b33cb => database_drop-28-53d67cbed634cff012dac41340bf6630} (100%) rename sql/hive/src/test/resources/golden/{create_like-9-a0ce573e299b66b8ce31da2890b318cb => database_drop-29-29d3232325eda40cbb0bd1786b7d070e} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 => database_drop-3-db64b724719d27c7f0db4f51f5c4edaa} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-0-366a4de0343396b9df03277f1098722c => database_drop-30-bbad0860316d8b9b1eed50d231f3ab5d} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 => database_drop-31-981f8e58caeae9cbbad3a113e3043de5} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f => database_drop-32-6c8e7590238b5aca1772721f0b914ece} (100%) rename sql/hive/src/test/resources/golden/{database_drop-5-b7cf74929eabe781b0db79ed1043dc24 => database_drop-33-2bc7864932f597bdf98bdc410b1a2d9c} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 => database_drop-34-491138bed44a70cb783bb2b531e1d82} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 => database_drop-35-9e0285d0596607cdadf75a763a543866} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-14-32251c08304629a3153e0b471de060c5 => database_drop-36-e66471f3f1bbe2d4b3b214e47793656d} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 => database_drop-37-f6410721e3125a89836817136306eac4} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 => database_drop-38-7d45d97adebe50917a94cbe232c112a8} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 => database_drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 => database_drop-4-f21dd011aeb417043ed58c03fd5c3bf0} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 => database_drop-40-10073fb74a5c792322fc52389997695b} (100%) rename sql/hive/src/test/resources/golden/{create_nested_type-3-ac452c9279877935983c37113898e53c => database_drop-41-7164c585e1ef4d9036ed4db275811084} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d => database_drop-42-c55cffbfc4d950363be97ccdb028faf3} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 => database_drop-43-cc6860c125b8b62450cb858c72716dc2} (100%) rename sql/hive/src/test/resources/golden/{exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee => database_drop-44-de81fd80132350aedcd9f0d9a212fd94} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea => database_drop-45-64e22634663e57153119340583e93651} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d => database_drop-46-eb28b907b605e51026f9902287e1d90d} (100%) rename sql/hive/src/test/resources/golden/{exim_02_part-1-baeaf0da490037e7ada642d23013075a => database_drop-47-44e4d2bdab2dfa4583da47281ed00ba3} (100%) rename sql/hive/src/test/resources/golden/{exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => database_drop-48-d419f4ff197d4291208c2028cd158909} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 => database_drop-5-d419f4ff197d4291208c2028cd158909} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 => database_drop-6-b7cf74929eabe781b0db79ed1043dc24} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de => database_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb => database_drop-8-b3980119a4b2d8f531951a940cba3697} (100%) rename sql/hive/src/test/resources/golden/{create_struct_table-1-2ca90a28a994405e6150c96f4a572294 => database_drop-9-97101266791d2b2c662bcde549422318} (100%) rename sql/hive/src/test/resources/golden/{create_union_table-0-8e765b54f15b948fc88392da69da283 => date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1} (100%) rename sql/hive/src/test/resources/golden/{create_union_table-1-aee4ce62fc2631423af0f569f4448353 => date_3-3-c26f0641e7cec1093273b258e6bf7120} (100%) rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee => date_4-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 => date_4-1-b84f7e931d710dcbe3c5126d998285a8} (100%) rename sql/hive/src/test/resources/golden/{create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 => date_4-2-6272f5e518f6a20bc96a5870ff315c4f} (100%) rename sql/hive/src/test/resources/golden/{create_view-1-c186ac1fe46117acb6fd452df15e0d92 => date_4-3-4a0e7bde447ef616b98e0f55d2886de0} (100%) rename sql/hive/src/test/resources/golden/{create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd => date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8} (100%) create mode 100644 sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 rename sql/hive/src/test/resources/golden/{create_view-11-2021c047203276dd2db51a56e672fcea => date_4-6-b84f7e931d710dcbe3c5126d998285a8} (100%) rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a => date_comparison-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 => date_comparison-1-69eec445bd045c9dc899fafa348d8495} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 => date_comparison-10-bcd987341fc1c38047a27d29dac6ae7c} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc => date_comparison-11-a9f2560c273163e11306d4f1dd1d9d54} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 => date_comparison-12-4a7bac9ddcf40db6329faaec8e426543} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-3-13e17ed811165196416f777cbc162592 => date_comparison-2-fcc400871a502009c8680509e3869ec1} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 => date_comparison-3-b8598a4d0c948c2ddcf3eeef0abf2264} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f => date_comparison-4-14d35f266be9cceb11a2ae09ec8b3835} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 => date_comparison-5-c8865b14d53f2c2496fb69ee8191bf37} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 => date_comparison-6-f2c907e64da8166a731ddc0ed19bad6c} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 => date_comparison-7-5606505a92bad10023ad9a3ef77eacc9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 => date_comparison-8-47913d4aaf0d468ab3764cc3bfd68eb} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-11-441306cae24618c49ec63445a31bf16b => date_comparison-9-1e5ce4f833b6fba45618437c8fb7643c} (100%) rename sql/hive/src/test/resources/golden/{create_view-12-420752b11848e29bce1c8ace7d3060fc => date_join1-2-e967e1ef6b209dfa5bdc60021dcb1964} (100%) rename sql/hive/src/test/resources/golden/{create_view-13-bff53e5df8356ac16f7b9b78b157e60a => date_serde-7-580096b3b48db26bea91b80e1e1b081a} (100%) rename sql/hive/src/test/resources/golden/{create_view-14-69162f2f22978113fea529d7fc7b78d3 => date_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 rename sql/hive/src/test/resources/golden/{create_view-15-ceebf4cb0dc23f517a444266bc8d2447 => decimal_4-2-85c3185beb011f5c1e6856fc773a7484} (100%) rename sql/hive/src/test/resources/golden/{create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 => decimal_4-3-1451d7491441c1632fd5f751876cce6e} (100%) rename sql/hive/src/test/resources/golden/{create_view-17-b3c1023d56f3439833c246e8bfd4502a => decimal_4-4-1bf9ff1d72a06c33885ba695adf2511d} (100%) rename sql/hive/src/test/resources/golden/{create_view-18-437d0a699b73c61044ebf8539feb14f6 => decimal_join-0-4668e9dee2cd7a32f2b7311d7cd35508} (100%) rename sql/hive/src/test/resources/golden/{create_view-2-d80dcd1271ab264292e9938f3162427c => decimal_join-1-5098974222b22a21ed847c7906df9313} (100%) delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab rename sql/hive/src/test/resources/golden/{create_view-20-56d203e4d0eb776bb4fa38409222b5b8 => delimiter-1-121ff21e6931a82235de8301118cbed8} (100%) delete mode 100644 sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 delete mode 100644 sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 delete mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 rename sql/hive/src/test/resources/golden/{create_view-21-3609711e61b5b8d241d0e839557bfd64 => disable_file_format_check-2-2d27f92dfced693fa3a68ecce5e2e838} (100%) rename sql/hive/src/test/resources/golden/{create_view-22-3bc364c0ee46900d2201d706d2d58d67 => disable_file_format_check-4-a2150709a6ff73326bdf4865dd124a23} (100%) rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 => disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a rename sql/hive/src/test/resources/golden/{exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee => disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b rename sql/hive/src/test/resources/golden/{read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b => disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b} (100%) rename sql/hive/src/test/resources/golden/{create_view-3-25ffe475d52d6c399acaf120dc02afe8 => disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11} (100%) delete mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c rename sql/hive/src/test/resources/golden/{create_view-4-87ed262d455e99ad45c909a2265a61b0 => disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782} (100%) create mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c rename sql/hive/src/test/resources/golden/{create_view-5-391caf27ff1589ec68d5f3bc4a27e711 => disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7} (100%) rename sql/hive/src/test/resources/golden/{exim_04_all_part-1-baeaf0da490037e7ada642d23013075a => distinct_stats-0-418ec894d08c33fd712eb358f579b7a0} (100%) rename sql/hive/src/test/resources/golden/{exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => distinct_stats-1-10987e425ba8ba8d9c01538f16eab970} (100%) rename sql/hive/src/test/resources/golden/{create_view-6-d8d0e830783c383e3c00e9de3919c409 => distinct_stats-2-a2d8f812612283b20ec3f1e92a263440} (100%) rename sql/hive/src/test/resources/golden/{create_view-7-50b35b8a1772becc96cff65bba1eaee7 => distinct_stats-3-a4397664f1f109ef0aa5ff36961b25b} (100%) rename sql/hive/src/test/resources/golden/{create_view-8-2ae18fc75eda9c3fe7d4e87829180805 => distinct_stats-4-e540680af4a857404a0cb7cabc1bcf31} (100%) rename sql/hive/src/test/resources/golden/{create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 => distinct_stats-5-32e9736bf27c1d2e4399a8125e14befc} (100%) rename sql/hive/src/test/resources/golden/{create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc => distinct_stats-6-297a53801744e6c4786e315e32c6189a} (100%) rename sql/hive/src/test/resources/golden/{create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 => distinct_stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d} (100%) rename sql/hive/src/test/resources/golden/{create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 => distinct_stats-8-ca16024e6f5399b1d035f5b9fd665163} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee => distinct_stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8} (100%) rename sql/hive/src/test/resources/golden/{create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 => drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-0-311fdd725609cd47ea1b859f706da41e => drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 => drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a => drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-2-3223504c97628a44b65736565c1dda32 => drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b => drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 => drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea} (100%) rename sql/hive/src/test/resources/golden/{exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee => drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 => drop_index_removes_partition_dirs-0-a99af48bbcbaba062e9bc387ae2b4975} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f => drop_index_removes_partition_dirs-1-e67a0f7ff61a97b2c49386890ea88c54} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 => drop_index_removes_partition_dirs-2-19915227905aab376d918b3cada85c25} (100%) rename sql/hive/src/test/resources/golden/{exim_05_some_part-1-baeaf0da490037e7ada642d23013075a => drop_index_removes_partition_dirs-3-e32b952789a115ff02201dfa618d92b2} (100%) rename sql/hive/src/test/resources/golden/{database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 => drop_index_removes_partition_dirs-4-58aff7fa05fba3a7549629a17e285036} (100%) rename sql/hive/src/test/resources/golden/{exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => drop_index_removes_partition_dirs-5-e32b952789a115ff02201dfa618d92b2} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee => drop_index_removes_partition_dirs-6-4642f8a18bf7409849f2e91d7a05f352} (100%) rename sql/hive/src/test/resources/golden/{database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 => drop_partitions_filter2-13-8117981303487dc4c4873356931ef26a} (100%) create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 rename sql/hive/src/test/resources/golden/{database_drop-7-693736836ccc99f6e2449b94efcfeeba => drop_partitions_filter2-15-67d75c72ea2d3982c55f3a850d93f83c} (100%) rename sql/hive/src/test/resources/golden/{database_drop-8-97101266791d2b2c662bcde549422318 => drop_partitions_filter2-7-f34625fd49a5e655cba3abb5cb8c5417} (100%) rename sql/hive/src/test/resources/golden/{database_drop-9-8db536f925bf0f5058f97897e145a661 => drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3} (100%) rename sql/hive/src/test/resources/golden/{date_2-2-cab14d992c53c106ab257fae52001e04 => drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff} (100%) rename sql/hive/src/test/resources/golden/{date_join1-2-894b6541812ac8b0abe2a24c966817d8 => drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-1-baeaf0da490037e7ada642d23013075a => drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299} (100%) rename sql/hive/src/test/resources/golden/{date_serde-7-a34279d8ebbadb78e925e8ed9c78947d => drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299} (100%) rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee => drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d} (100%) rename sql/hive/src/test/resources/golden/{ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 => dynamic_partition_skip_default-0-3cd14bc5b126ff8b337c4abc09134260} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 => dynamic_partition_skip_default-2-ce929ee6a92b81d8080ca322c1c38a4b} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd => dynamic_partition_skip_default-3-f3a5a998099b756a21cf9122a15b09d5} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 => dynamic_partition_skip_default-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-2-80fc87cab17ceffea334afbb230a6653 => dynamic_partition_skip_default-5-fed732d2e7d94a4fc02e7694f9f9a39c} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 => dynamic_partition_skip_default-6-adab6a0187003ab7ee6f217c9e409d91} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-0-3c8ecb1468952afe028596c65d587bee => dynamic_partition_skip_default-8-adab6a0187003ab7ee6f217c9e409d91} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-1-868e124edc1581325bd0fd10235a126b => dynamic_partition_skip_default-9-4b099f87e221b8fd5c0d0d4a97c0d146} (100%) delete mode 100644 sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 delete mode 100644 sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf delete mode 100644 sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 delete mode 100644 sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 delete mode 100644 sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 delete mode 100644 sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 delete mode 100644 sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 delete mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d delete mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 delete mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d delete mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 rename sql/hive/src/test/resources/golden/{decimal_2-12-d63b5ea25e27852413132db4d9bfb035 => explain_rearrange-0-6f7c8515c354fb050829ebd66413425} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-2-6cc742523b3574e59ca21dad30f2d506 => explain_rearrange-1-378d42317b39c6519f15bd2f99c5ddc4} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e => explain_rearrange-10-3f2680208772a0e51aefc4ef5604dddf} (100%) rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a => explain_rearrange-11-f2ca12a948fd9b5b842168e7c7d7b768} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-31-3c8ecb1468952afe028596c65d587bee => explain_rearrange-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e} (100%) rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 => explain_rearrange-13-5baad22ed7efa18d73eb8349e57cf331} (100%) rename sql/hive/src/test/resources/golden/{decimal_3-0-90cd495a00051a0631b2021dbb9a4aef => explain_rearrange-14-490d6253b73064ce403e4d04a8bc18f3} (100%) rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee => explain_rearrange-2-24ca942f094b14b92086305cc125e833} (100%) rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a => explain_rearrange-3-3b0f76816be2c1b18a2058027a19bc9f} (100%) rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 => explain_rearrange-4-86473a0498e4361e4db0b4a22f2e8571} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee => explain_rearrange-5-d0ec6d66ff349db09fd455eec149efdb} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a => explain_rearrange-6-cda81d86d127fca0e2fbc2161e91400d} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 => explain_rearrange-7-5b13cfa4b730e38ef2794c1532968e04} (100%) rename sql/hive/src/test/resources/golden/{decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 => explain_rearrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12} (100%) rename sql/hive/src/test/resources/golden/{exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee => explain_rearrange-9-73b9ac83dbc9874dc9379ad4364d40ac} (100%) rename sql/hive/src/test/resources/golden/{decimal_3-12-d495d7178707ba55dcc01b9bb3398792 => filter_join_breaktask2-4-fe8b55e4e4098d7a2662338783a50306} (100%) rename sql/hive/src/test/resources/golden/{decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 => filter_join_breaktask2-5-cf724251613216ec54f8ac2e6b9b92fd} (100%) rename sql/hive/src/test/resources/golden/{decimal_4-2-945542ec888136afaece8d7a5e20d52d => filter_join_breaktask2-6-8c782ae8f8245bdbe90d068a6b577d1e} (100%) rename sql/hive/src/test/resources/golden/{exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a => groupby12-0-67e864faaff4c6b2a8e1c9fbd188bb66} (100%) rename sql/hive/src/test/resources/golden/{decimal_4-3-399140971a10a5a0cc6a8c97a4635e => groupby12-1-13ab74a58da514fe01dbeda0c3e79883} (100%) rename sql/hive/src/test/resources/golden/{decimal_4-4-81b37675c09ed874497325ae13233e5c => groupby12-2-fd150794945892f3c926a1881cd819f4} (100%) rename sql/hive/src/test/resources/golden/{decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 => groupby12-3-8018bb917a0706925c14421ec2761663} (100%) create mode 100644 sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f rename sql/hive/src/test/resources/golden/{decimal_join-1-c1524f17ee815171055a67ddc2f9de4e => groupby1_limit-3-607512a8dd16cd9ddde561eeabfa51db} (100%) delete mode 100644 sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b delete mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b delete mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb rename sql/hive/src/test/resources/golden/{decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e => groupby2_limit-1-a56d6499aef913e11ef599ac8b4f2a25} (100%) delete mode 100644 sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 create mode 100644 sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a delete mode 100644 sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b delete mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 delete mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c delete mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f delete mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f delete mode 100644 sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f delete mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f delete mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 rename sql/hive/src/test/resources/golden/{decimal_precision-1-3f4119830536c92e5ccd76be0259e110 => groupby7_noskew_multi_single_reducer-8-ab2390a3b0ba76907a7bee390b2924a1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b delete mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d delete mode 100644 sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 rename sql/hive/src/test/resources/golden/{decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 => groupby_sort_2-5-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{decimal_precision-15-5c49f041326bc5a9e936910094f190ce => groupby_sort_3-5-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c => groupby_sort_4-5-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{delimiter-1-d9e405c4107da8af78fcacb83a667b41 => groupby_sort_5-13-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb => groupby_sort_5-20-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 => groupby_sort_5-5-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{describe_database_json-10-1afddec8522bd69f496b15980600a6e1 => groupby_sort_6-9-591e03d1cfc10821a601498df1ed6675} (100%) rename sql/hive/src/test/resources/golden/{describe_database_json-6-1afddec8522bd69f496b15980600a6e1 => groupby_sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc} (100%) rename sql/hive/src/test/resources/golden/{describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 => groupby_sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc} (100%) rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 => groupby_sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc} (100%) delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa => groupby_sort_test_1-6-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf delete mode 100644 sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d delete mode 100644 sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad delete mode 100644 sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f delete mode 100644 sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e delete mode 100644 sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf delete mode 100644 sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca delete mode 100644 sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe delete mode 100644 sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f delete mode 100644 sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b delete mode 100644 sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d delete mode 100644 sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b delete mode 100644 sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c delete mode 100644 sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 rename sql/hive/src/test/resources/golden/{exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 => index_serde-0-db64b724719d27c7f0db4f51f5c4edaa} (100%) rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 => index_serde-1-6560d12b69d55e5297a145ebc4bb0cb3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 delete mode 100644 sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 => index_serde-10-c85e061ea9c5b90ca69b7450faad14b6} (100%) create mode 100644 sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b delete mode 100644 sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 rename sql/hive/src/test/resources/golden/{describe_pretty-0-f34ca99310bf1d4793cf64423c024ad => index_serde-12-309e916d683a1a12ab62565697cb0046} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 rename sql/hive/src/test/resources/golden/{describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 => index_serde-13-d590fd7cb9d433143de490d75686dd4} (100%) create mode 100644 sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 delete mode 100644 sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c delete mode 100644 sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 rename sql/hive/src/test/resources/golden/{describe_pretty-9-e382a994134aefcd2652b57af9195644 => index_serde-3-ebab588c84a7a29f03b41dcd98132229} (100%) rename sql/hive/src/test/resources/golden/{describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a => index_serde-4-afcf2a156ccd4f79a0489b4593908d79} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 create mode 100644 sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 delete mode 100644 sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 delete mode 100644 sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae rename sql/hive/src/test/resources/golden/{describe_syntax-1-4f3d3497418242124113538edab45df7 => index_serde-6-e6ff4b23b7f102e359afb4d53a1dedc3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 rename sql/hive/src/test/resources/golden/{exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee => index_serde-7-c9d7dcde469d3b9a66965a64dd15e4ae} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 rename sql/hive/src/test/resources/golden/{exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a => index_serde-8-3b03210f94ec40db9ab02620645014d1} (100%) rename sql/hive/src/test/resources/golden/{exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 => index_serde-9-35f48c7d6fa164bb84643657bc9280a8} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 delete mode 100644 sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e delete mode 100644 sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 delete mode 100644 sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd delete mode 100644 sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 delete mode 100644 sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf delete mode 100644 sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 delete mode 100644 sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 delete mode 100644 sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 delete mode 100644 sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 delete mode 100644 sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 delete mode 100644 sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc rename sql/hive/src/test/resources/golden/{describe_syntax-2-b198700c0129910d6205ef063ee83d5a => input19-1-f32df514de8156b5f5b435eea2c9be40} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 rename sql/hive/src/test/resources/golden/{describe_syntax-3-458d6aaffeee94997f67a43b88382106 => input21-1-c45ad493e95150b580be778da6065f36} (100%) rename sql/hive/src/test/resources/golden/{describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 => input22-1-b663ec84da3f9d9b9594ea2da81b1442} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 delete mode 100644 sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c delete mode 100644 sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab delete mode 100644 sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee delete mode 100644 sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 delete mode 100644 sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 delete mode 100644 sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 delete mode 100644 sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 delete mode 100644 sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f delete mode 100644 sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe delete mode 100644 sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f delete mode 100644 sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 delete mode 100644 sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d delete mode 100644 sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 delete mode 100644 sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e delete mode 100644 sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 delete mode 100644 sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 delete mode 100644 sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 delete mode 100644 sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 delete mode 100644 sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 delete mode 100644 sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 delete mode 100644 sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 delete mode 100644 sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 delete mode 100644 sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d delete mode 100644 sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 delete mode 100644 sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e delete mode 100644 sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 delete mode 100644 sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee delete mode 100644 sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 delete mode 100644 sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e delete mode 100644 sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 delete mode 100644 sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 rename sql/hive/src/test/resources/golden/{describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 => input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 rename sql/hive/src/test/resources/golden/{describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 => input4-2-b663ec84da3f9d9b9594ea2da81b1442} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 rename sql/hive/src/test/resources/golden/{describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb => input40-1-a8adb8ae1d13607851431a1baf7578ba} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 delete mode 100644 sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e rename sql/hive/src/test/resources/golden/{describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 => input40-4-fdeea6b676c670b17c8d91e24a97a127} (100%) rename sql/hive/src/test/resources/golden/{disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 => input40-6-6651f53efc5d03ed2d43b9d7aecc0002} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 delete mode 100644 sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a delete mode 100644 sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 delete mode 100644 sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f delete mode 100644 sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca delete mode 100644 sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 delete mode 100644 sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 delete mode 100644 sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab rename sql/hive/src/test/resources/golden/{disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb => input4_cb_delim-1-353d2238b781a117888a67bb7b2b2537} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 delete mode 100644 sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d delete mode 100644 sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af delete mode 100644 sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea delete mode 100644 sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 delete mode 100644 sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad delete mode 100644 sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 rename sql/hive/src/test/resources/golden/{disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 => inputddl6-1-572c1abb70f09726d1ba77bdc884597b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf rename sql/hive/src/test/resources/golden/{disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 => inputddl6-2-3a4def4b370f75c5fcc1174626490363} (100%) delete mode 100644 sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c rename sql/hive/src/test/resources/golden/{disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 => inputddl7-1-7195712efb4910294f63303ebce24453} (100%) delete mode 100644 sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 delete mode 100644 sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d rename sql/hive/src/test/resources/golden/{drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c => inputddl7-10-7c9248b56948716913d332bd712d69bd} (100%) rename sql/hive/src/test/resources/golden/{drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 => inputddl7-4-68715ba2c11220be62394c86453e6d54} (100%) delete mode 100644 sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 delete mode 100644 sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 => inputddl7-7-59dd2d2556769e19bdc0a444f40f8a71} (100%) rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 => insert2_overwrite_partitions-3-86653b3af59df59f225ee00ff5fc119f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 delete mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 => insert2_overwrite_partitions-4-e81d45a5bec5642ec4b762f1c1a482af} (100%) delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab delete mode 100644 sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 delete mode 100644 sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a delete mode 100644 sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b delete mode 100644 sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 delete mode 100644 sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 => join_1to1-1-789b2636cfb6a08965e0bd190e419762} (100%) rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 => join_1to1-3-6228e662e573a00ed04550d049d97a3b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 => join_array-2-16840a0266cad03a1a0b134d105b854f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 => join_array-3-a6ca6b64324596831033fdfe5b63a942} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 delete mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 => join_casesensitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc rename sql/hive/src/test/resources/golden/{enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 => join_casesensitive-3-93300f4a9242fa2804a5b368538d83f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 rename sql/hive/src/test/resources/golden/{enforce_order-1-633034e3d966737cecf2befc5df1e35d => join_filters-1-bee6095f42de6a16708c2f9addc1b9bd} (100%) rename sql/hive/src/test/resources/golden/{enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f => join_filters-46-268d8fb3cb9b04eb269fe7ec40a24dfe} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b delete mode 100644 sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 rename sql/hive/src/test/resources/golden/{enforce_order-4-3136edd49e681ea21aa35d0836eab65 => join_filters-47-6dc6866a65c74d69538b776b41b06c16} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c rename sql/hive/src/test/resources/golden/{enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 => join_filters-48-e884480a0f7273d3e2f2de2ba46b855c} (100%) rename sql/hive/src/test/resources/golden/{enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 => join_filters-49-98fd86aea9cacaa82d43c7468109dd33} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 delete mode 100644 sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce delete mode 100644 sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 delete mode 100644 sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 rename sql/hive/src/test/resources/golden/{escape1-2-395d5a528c5e7235a48b4ac90938e2d6 => join_hive_626-3-4a2f2f2858540afea9a195b5322941ee} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 rename sql/hive/src/test/resources/golden/{escape1-3-4267651148da591da38737028fdbd80 => join_hive_626-4-4bb73b33747da4ed852df381b7b45a71} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 rename sql/hive/src/test/resources/golden/{escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 => join_hive_626-5-c6a3ae6f3539ab48b996060fb51d8ebe} (100%) rename sql/hive/src/test/resources/golden/{escape1-5-70729c3d79ded87e884c176138174645 => join_nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 rename sql/hive/src/test/resources/golden/{escape1-8-910536a438eec89c78bd611b3c4bb7e0 => join_nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 rename sql/hive/src/test/resources/golden/{escape2-10-13884d58efe80bd24862b3c54cb57c6e => join_nulls-41-6dc6866a65c74d69538b776b41b06c16} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c rename sql/hive/src/test/resources/golden/{escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe => join_nulls-42-e884480a0f7273d3e2f2de2ba46b855c} (100%) rename sql/hive/src/test/resources/golden/{escape2-5-fd0c88ad6ad131a16d1b78adbea65800 => join_nulls-43-98fd86aea9cacaa82d43c7468109dd33} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 rename sql/hive/src/test/resources/golden/{escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 => join_nullsafe-2-5bb63fafa390b1d4c20e225a8a648dcf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 rename sql/hive/src/test/resources/golden/{escape2-7-70729c3d79ded87e884c176138174645 => join_nullsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 => join_nullsafe-21-24332b9390108fb3379e1acc599293a1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 rename sql/hive/src/test/resources/golden/{exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 => join_nullsafe-22-4be80634a6bd916e3ebd60a124f0a48e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 rename sql/hive/src/test/resources/golden/{exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd => join_nullsafe-23-e4425d56be43c21124d95160653ce0ac} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 => join_reorder2-4-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 => join_reorder2-5-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 rename sql/hive/src/test/resources/golden/{exim_02_part-4-88b581725ecdd603117a1706ab9c34dc => join_reorder2-6-8eb53fb8f05a43ee377aa1c927857e7c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 => join_reorder2-7-5f4cfbbe53c5e808ee08b26514272034} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd => join_reorder3-4-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 => join_reorder3-5-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 rename sql/hive/src/test/resources/golden/{exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc => join_reorder3-6-8eb53fb8f05a43ee377aa1c927857e7c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec rename sql/hive/src/test/resources/golden/{exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a => join_reorder3-7-5f4cfbbe53c5e808ee08b26514272034} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 => join_reorder4-3-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 => join_reorder4-4-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 => join_reorder4-5-8eb53fb8f05a43ee377aa1c927857e7c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 => join_star-10-a9e579038e3d4826fdae475d7058ab82} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 => join_star-11-72730ecdad9c0fd4c6ce64a0cb89fb74} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae delete mode 100644 sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 => join_star-12-f581d6d305d652cd0f4e4fa912eb578d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 => join_star-13-7268564732cbb7489248f9d818f80c14} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 => join_star-14-2ee0fcf000f8687fc8941bf212477e57} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 delete mode 100644 sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 => join_star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 rename sql/hive/src/test/resources/golden/{exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 => join_star-8-c3d53a4daab9614a09870dc8e9571f74} (100%) rename sql/hive/src/test/resources/golden/{exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc => join_star-9-3f7ccccc2488de5f33a38cb3cc3eb628} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 delete mode 100644 sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 delete mode 100644 sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db delete mode 100644 sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b delete mode 100644 sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 delete mode 100644 sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 delete mode 100644 sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 delete mode 100644 sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 delete mode 100644 sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b delete mode 100644 sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 delete mode 100644 sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef delete mode 100644 sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d delete mode 100644 sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 delete mode 100644 sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b rename sql/hive/src/test/resources/golden/{exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a => lateral_view_ppd-9-dc6fea663d875b082d38bd326d21cd95} (100%) delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin-5-9c307c0559d735960ce77efa95b2b17b rename sql/hive/src/test/resources/golden/{exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 => leftsemijoin-5-aba449db0d4fe6dc9771426e102bb543} (100%) delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin-6-82921fc96eef547ec0f71027ee88298c rename sql/hive/src/test/resources/golden/{exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 => leftsemijoin-6-9f50dce576b019c0be997055b8876621} (100%) delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 rename sql/hive/src/test/resources/golden/{exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 => leftsemijoin-7-fff6ca40e6048d52dc2d3afc68e8353e} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc => leftsemijoin_mr-1-5b2e555868faa404ea09928936178181} (100%) delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin_mr-1-aa3f07f028027ffd13ab5535dc821593 delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 rename sql/hive/src/test/resources/golden/{exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a => leftsemijoin_mr-3-c019cb2a855138da0d0b1e5c67cd6354} (100%) delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 delete mode 100644 sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f rename sql/hive/src/test/resources/golden/{exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee => literal_double-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 => literal_double-1-10ef1098e35d900983be3814de8f974f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 create mode 100644 sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 rename sql/hive/src/test/resources/golden/{exim_12_external_location-1-baeaf0da490037e7ada642d23013075a => literal_ints-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 rename sql/hive/src/test/resources/golden/{exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 => literal_ints-1-5ffd1b49cdda4149aef2c61c53a56890} (100%) delete mode 100644 sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 create mode 100644 sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 rename sql/hive/src/test/resources/golden/{exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => literal_string-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 delete mode 100644 sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 => literal_string-1-9b48d41e5c9e41ddc070e2fd31ace15} (100%) create mode 100644 sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-10-c66fea7ad025cd1f513a98a4cc1036b1 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-11-8b6be78ae0527e5b7efd7db758966853 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-9-a646fd106fe73b8753fe3fee86d56ebf delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-8-c32770da4784bfea4d0dd77fdcba4f0a delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-10-297cf42ec45b5aa78b80d9c35a79555a delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-11-24618a43c4656b72f81683b45708045e create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-9-704bd110d9aaa2ac678b7fbf645abdb9 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-9-6954f5fc8dd82ca2c076ab8bcdbc148 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb delete mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 delete mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 delete mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 delete mode 100644 sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc => load_file_with_space_in_the_name-1-d19201e2fcaee4d451292bd740e6c637} (100%) rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a => load_file_with_space_in_the_name-2-ad8795e50f5998ea1d2eb64a0c02e6e5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 delete mode 100644 sql/hive/src/test/resources/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 delete mode 100644 sql/hive/src/test/resources/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 delete mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc delete mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 delete mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 delete mode 100644 sql/hive/src/test/resources/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 delete mode 100644 sql/hive/src/test/resources/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 delete mode 100644 sql/hive/src/test/resources/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 delete mode 100644 sql/hive/src/test/resources/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 delete mode 100644 sql/hive/src/test/resources/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 => loadpart1-1-6cc94d19c536a996592629f7c82c2ac9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e delete mode 100644 sql/hive/src/test/resources/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 delete mode 100644 sql/hive/src/test/resources/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 delete mode 100644 sql/hive/src/test/resources/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 delete mode 100644 sql/hive/src/test/resources/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 delete mode 100644 sql/hive/src/test/resources/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 delete mode 100644 sql/hive/src/test/resources/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 delete mode 100644 sql/hive/src/test/resources/golden/lock2-3-27ad2962fed131f51ba802596ba37278 delete mode 100644 sql/hive/src/test/resources/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a delete mode 100644 sql/hive/src/test/resources/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca delete mode 100644 sql/hive/src/test/resources/golden/lock3-0-27ad2962fed131f51ba802596ba37278 delete mode 100644 sql/hive/src/test/resources/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a delete mode 100644 sql/hive/src/test/resources/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 delete mode 100644 sql/hive/src/test/resources/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac delete mode 100644 sql/hive/src/test/resources/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe delete mode 100644 sql/hive/src/test/resources/golden/lock4-1-27ad2962fed131f51ba802596ba37278 delete mode 100644 sql/hive/src/test/resources/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a delete mode 100644 sql/hive/src/test/resources/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 delete mode 100644 sql/hive/src/test/resources/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac delete mode 100644 sql/hive/src/test/resources/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 => mapjoin_mapjoin-4-5166a5b9d30dfacbe33dd909c0df6310} (100%) delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-5ee898cab883074f3297198c52445ee4 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 => mapjoin_mapjoin-6-dca8c08a235b45d1cdcb94e363afb17} (100%) create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-7-fddbdea343a9ddb5f8dedc18147640b7 rename sql/hive/src/test/resources/golden/{exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee => mapjoin_mapjoin-8-2be637ed4f6146e8525ae1a863e72736} (100%) create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-9-c47698bac140454637a999e583941ce7 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd => mapjoin_subquery2-6-9bf06af695892b0d7067d5b30e0b2425} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 => mapjoin_subquery2-7-c6b0cdb137f13f8362c0c49c544151a4} (100%) delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc => mapjoin_subquery2-8-131ae5ecfff2733b04bdfada0108cf40} (100%) delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e delete mode 100644 sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 delete mode 100644 sql/hive/src/test/resources/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 delete mode 100644 sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 delete mode 100644 sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 delete mode 100644 sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 delete mode 100644 sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a delete mode 100644 sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d delete mode 100644 sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af delete mode 100644 sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d delete mode 100644 sql/hive/src/test/resources/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 delete mode 100644 sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 delete mode 100644 sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e delete mode 100644 sql/hive/src/test/resources/golden/mi-3-b66a495f7bdf106a7886b72267b8659d delete mode 100644 sql/hive/src/test/resources/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c delete mode 100644 sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc delete mode 100644 sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 delete mode 100644 sql/hive/src/test/resources/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 delete mode 100644 sql/hive/src/test/resources/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 rename sql/hive/src/test/resources/golden/{exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a => multiMapJoin2-28-c14792ccac2ca64e3e9e21af4fd12d2c} (100%) rename sql/hive/src/test/resources/golden/{exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => multiMapJoin2-29-b9d963d24994c47c3776dda6f7d3881f} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a => multiMapJoin2-30-6d1c7f7014fc709148b0f401c5f23cb3} (100%) create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-31-2e9c4d5e2bd709e96f311488ada116b0 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f delete mode 100644 sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 delete mode 100644 sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 delete mode 100644 sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 => nullgroup3-1-61ead7f73d525e0d9e21beba91a3d39e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 => nullgroup3-12-61ead7f73d525e0d9e21beba91a3d39e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 rename sql/hive/src/test/resources/golden/{exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 => nullgroup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb rename sql/hive/src/test/resources/golden/{exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd => nullgroup3-18-113e2b587784d54d2a5b5f091389397e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 rename sql/hive/src/test/resources/golden/{exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 => nullgroup3-19-77de4b2d65eee228848625cdd422178d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 rename sql/hive/src/test/resources/golden/{exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd => nullgroup3-2-a5e12cfbc1799ce9fa9628d81b8c0b06} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb rename sql/hive/src/test/resources/golden/{exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 => nullgroup3-6-113e2b587784d54d2a5b5f091389397e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 rename sql/hive/src/test/resources/golden/{exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd => nullgroup3-7-77de4b2d65eee228848625cdd422178d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 => nullgroup5-1-642e12a05bf01a6716bfa15ed0012629} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd => nullgroup5-3-35517978e09aa1bd1d15f51d11e08fd5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c delete mode 100644 sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 rename sql/hive/src/test/resources/golden/{exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 => nullscript-1-f07dfd6ef687e038083deca5941d8174} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 rename sql/hive/src/test/resources/golden/{exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc => nullscript-2-bb1abcf2f4a2a5cd5c058104901627bb} (100%) delete mode 100644 sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc delete mode 100644 sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d delete mode 100644 sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d delete mode 100644 sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead delete mode 100644 sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c delete mode 100644 sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae delete mode 100644 sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d delete mode 100644 sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a delete mode 100644 sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae delete mode 100644 sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 delete mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 rename sql/hive/src/test/resources/golden/{exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a => orc_dictionary_threshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 delete mode 100644 sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 delete mode 100644 sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b delete mode 100644 sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 delete mode 100644 sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 delete mode 100644 sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 delete mode 100644 sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 rename sql/hive/src/test/resources/golden/{exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 => partcols1-1-b562ff3e36de23a686d122967a1f91c8} (100%) rename sql/hive/src/test/resources/golden/{exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 => partition_date-1-6b0952309c3ebdd6dcb7066891d1bd74} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f rename sql/hive/src/test/resources/golden/{orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 => partition_date-11-a01c3791f59e819c750e213b6c65d084} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 create mode 100644 sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee => partition_date-14-3f187dad9a2fdfc6f7a3566309016f9c} (100%) rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a => partition_date-15-e4366325f3a0c4a8e92be59f4de73fce} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac create mode 100644 sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 create mode 100644 sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 create mode 100644 sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d rename sql/hive/src/test/resources/golden/{exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 => partition_date-2-220048240a7050a98411ddbc6b1f82cf} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba create mode 100644 sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 rename sql/hive/src/test/resources/golden/{type_cast_1-1-53a667981ad567b2ab977f67d65c5825 => partition_date-22-fe277bd0a30e016826d242fd5bd61714} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 rename sql/hive/src/test/resources/golden/{exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc => partition_date-24-7ec1f3a845e2c49191460e15af30aa30} (100%) rename sql/hive/src/test/resources/golden/{exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a => partition_date-3-f8887dd18c21bf0306e293d463b3e1d7} (100%) rename sql/hive/src/test/resources/golden/{exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 => partition_date-4-f7c20c208e052305c1710a5e1e80c5c8} (100%) rename sql/hive/src/test/resources/golden/{exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 => partition_date-5-59355f4e222dcd4f77f51d15bd896f11} (100%) rename sql/hive/src/test/resources/golden/{exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 => partition_date-6-416ab10ac818d432cd3a81870d6e5164} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc create mode 100644 sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 create mode 100644 sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 rename sql/hive/src/test/resources/golden/{exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc => partition_type_check-11-60aad884be613d18d3b89fca3b90dc94} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a rename sql/hive/src/test/resources/golden/{exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a => partition_type_check-2-3a6bb204d9524299f28adf1cc35d6f4d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f delete mode 100644 sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 rename sql/hive/src/test/resources/golden/{exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 => partition_type_check-6-663051c7106b8ee6913ca1b007157941} (100%) rename sql/hive/src/test/resources/golden/{exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 => partition_varchar1-2-bca5c9edccc3a84e0a9ef92ebcbe746} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 delete mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 rename sql/hive/src/test/resources/golden/{exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 => partition_varchar1-3-9cec4b1d156f5a9cb587470b98de15} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef rename sql/hive/src/test/resources/golden/{exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc => partition_varchar1-4-e9c4530e270db6d44cc54292e4eff680} (100%) rename sql/hive/src/test/resources/golden/{exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a => partition_varchar1-5-661aefd18c44c1eb02c2aaf8fab59f73} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c delete mode 100644 sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 delete mode 100644 sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 delete mode 100644 sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 delete mode 100644 sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 => ppr_pushdown-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b rename sql/hive/src/test/resources/golden/{exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 => ppr_pushdown-1-855b235f9c760ba9d6d0200bfd1ad08b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 rename sql/hive/src/test/resources/golden/{exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 => ppr_pushdown-10-2957fd9b211cee5f0372525a1de55c19} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 => ppr_pushdown-11-b8d6f0ffc8294497c792b26958adee45} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc => ppr_pushdown-12-ab1b9a0cdd9586c96d8856a9d632b89c} (100%) rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 => ppr_pushdown-13-2c316c67fd494d878fbbea107d283c3b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 => ppr_pushdown-14-53b4be82a1538844d03b200429efa02b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc => ppr_pushdown-15-71ab93d38ed2908069091c7c8cc0aba1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a => ppr_pushdown-16-855cb54d28034fdb20a3615ee0918d63} (100%) rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 => ppr_pushdown-17-d3ccf2722a8b7281fcee61b2544772c8} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 => ppr_pushdown-18-fcd10fbfc55a2c0aa843fe618f9613c6} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 => ppr_pushdown-19-ff36e3978571ac05f11e8322c024e4b6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 => ppr_pushdown-2-f689aaf15b08c433d1e93ce977a9b6d5} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a => ppr_pushdown-20-481005cf034ef3d7b998da32eb82aa9a} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 => ppr_pushdown-21-9073825e8b9804331f780980d1f9fa92} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d => ppr_pushdown-22-9cb28f0d3a434c9d1ab039192906ec9d} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d => ppr_pushdown-23-678f50025924fee7b59f66e2abdb472d} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 => ppr_pushdown-24-4d5bfa800ba434f464b07bf9b5d39f20} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 => ppr_pushdown-25-22663f09ea1c1bc303756067e84df5a7} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 => ppr_pushdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 => ppr_pushdown-27-8065c18e387fd8bffae19a80af8dc1d4} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-27-b72de558c88ae91460989938000e0d27 => ppr_pushdown-28-b72de558c88ae91460989938000e0d27} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 => ppr_pushdown-29-7217735d357770af4ffee730e4e9add4} (100%) rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc => ppr_pushdown-3-5eede4874e678021938683fc2f5dc900} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 rename sql/hive/src/test/resources/golden/{ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea => ppr_pushdown-30-630e2f7918b7727fc4ca057fa21e2eea} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 => ppr_pushdown-31-2e062414293b643ea4d7f6df92f939e4} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-31-5eba4bf10315099129eae319d73636cf => ppr_pushdown-32-5eba4bf10315099129eae319d73636cf} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b => ppr_pushdown-33-35af0585a4f98bc222c786688cb6de6b} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 => ppr_pushdown-34-a5594625510703427ab8bae5d0563f73} (100%) rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a => ppr_pushdown-4-c5c542f8ee81cd0afd44e67fc7b4d306} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 => ppr_pushdown-5-f54bebec398f0fdfdbc0393123dba234} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 => ppr_pushdown-6-78af12432bcbf99d4a0d41c25f964de} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 => ppr_pushdown-7-58724fbe96a0b3deceef20c8cc5e318d} (100%) rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd => ppr_pushdown-8-14570f946e75924d7926c809485951d1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 => ppr_pushdown-9-3ba325662296fc455f07f1c835495e4c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 rename sql/hive/src/test/resources/golden/{exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee => ppr_pushdown2-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 => ppr_pushdown2-1-855b235f9c760ba9d6d0200bfd1ad08b} (100%) create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd => ppr_pushdown2-11-ab3e46183487096745d7d90e3020e94c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 => ppr_pushdown2-12-b10b9e14f5a7a7a92c0c68df6dbc656a} (100%) rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc => ppr_pushdown2-13-154f9859bd0822e287fbfdff12fd45ff} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 => ppr_pushdown2-14-cf4a30b5c8329d8d79ddf762f318fbb3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd => ppr_pushdown2-2-ffa167b63d612a4986d02f5c0623ea7b} (100%) create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c rename sql/hive/src/test/resources/golden/{filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 => ppr_pushdown2-3-c7753746c190414723d66a8f876499c7} (100%) create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 rename sql/hive/src/test/resources/golden/{filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba => ppr_pushdown2-6-b5a2518af801f95fe52a75dfc1d3e867} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 rename sql/hive/src/test/resources/golden/{filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 => ppr_pushdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b} (100%) rename sql/hive/src/test/resources/golden/{groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 => ppr_pushdown2-8-4507a3f200b3ce384191c91acd324dc7} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 delete mode 100644 sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 delete mode 100644 sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a delete mode 100644 sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 delete mode 100644 sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f rename sql/hive/src/test/resources/golden/{groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 => progress_1-2-b6c8c8fc9df98af4dead5efabf5f162c} (100%) rename sql/hive/src/test/resources/golden/{exim_15_external_part-1-baeaf0da490037e7ada642d23013075a => quote2-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 delete mode 100644 sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 => quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118} (100%) create mode 100644 sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e delete mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 delete mode 100644 sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b delete mode 100644 sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 delete mode 100644 sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 delete mode 100644 sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db delete mode 100644 sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 delete mode 100644 sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 delete mode 100644 sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 delete mode 100644 sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 delete mode 100644 sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa delete mode 100644 sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 delete mode 100644 sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d delete mode 100644 sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 delete mode 100644 sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 delete mode 100644 sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 delete mode 100644 sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 delete mode 100644 sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef delete mode 100644 sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f delete mode 100644 sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 delete mode 100644 sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 create mode 100644 sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 => serde_regex-2-e84d30fcc6cf11e82b54ea63e7d1d611} (100%) rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 => serde_regex-3-3ee9e78ff563d6b48741a41885f92c81} (100%) delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 => serde_regex-7-bf456bcf6be7334488424dfeadf27d75} (100%) rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 => serde_regex-8-a4cf34af32b83e40e5c8b6d083938b54} (100%) delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 => serde_regex-9-f1175f3322abec6f258dd49a5905bce0} (100%) delete mode 100644 sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 rename sql/hive/src/test/resources/golden/{stats4-2-463330cf55370dbe92d6ed74ef91302 => show_partitions-1-e69b801a3c6c5f6692050bcdb0e31db9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 delete mode 100644 sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca create mode 100644 sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 delete mode 100644 sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 rename sql/hive/src/test/resources/golden/{stats4-11-ea921e0af59a4940a11c94143b1c4b32 => show_partitions-3-a1bde7c2c040b4d45ddceac9983c2ca} (100%) create mode 100644 sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce delete mode 100644 sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c delete mode 100644 sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c delete mode 100644 sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe delete mode 100644 sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab delete mode 100644 sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c delete mode 100644 sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d delete mode 100644 sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c delete mode 100644 sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 delete mode 100644 sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 delete mode 100644 sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a => skewjoinopt13-4-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 => skewjoinopt13-6-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 rename sql/hive/src/test/resources/golden/{groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 => skewjoinopt13-8-8eb53fb8f05a43ee377aa1c927857e7c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 rename sql/hive/src/test/resources/golden/{groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a => skewjoinopt18-4-abf4b7f158999af331d5dbfddf32fa68} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 => skewjoinopt18-8-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 => skewjoinopt9-4-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 => skewjoinopt9-6-ade68a23d7b1a4f328623bb5a0f07488} (100%) rename sql/hive/src/test/resources/golden/{groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 => smb_mapjoin_1-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 => smb_mapjoin_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 => smb_mapjoin_1-5-6d835f651b099615df163be284e833de} (100%) rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 => smb_mapjoin_10-3-68d65d622e45f86d4a6c7d1d09ef823b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 => smb_mapjoin_10-4-d31ad2289181131982ef3e9cd8c6386e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b => smb_mapjoin_10-5-f7fb003fa65cadcd0b13cbdd7b355988} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 => smb_mapjoin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab => smb_mapjoin_2-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 => smb_mapjoin_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 => smb_mapjoin_2-5-6d835f651b099615df163be284e833de} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc => smb_mapjoin_25-10-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 => smb_mapjoin_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f => smb_mapjoin_25-12-6d835f651b099615df163be284e833de} (100%) rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 => smb_mapjoin_3-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 => smb_mapjoin_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea => smb_mapjoin_3-5-6d835f651b099615df163be284e833de} (100%) rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b => smb_mapjoin_4-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 => smb_mapjoin_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_position-1-6b06902de5c0ca13cebe03018d86f447 => smb_mapjoin_4-5-6d835f651b099615df163be284e833de} (100%) rename sql/hive/src/test/resources/golden/{groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 => smb_mapjoin_5-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_position-13-9ee2150594ad2eece6ee14424155d396 => smb_mapjoin_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_position-2-627bb7be9c0edb5ba4c677912800d364 => smb_mapjoin_5-5-6d835f651b099615df163be284e833de} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb rename sql/hive/src/test/resources/golden/{groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac => smb_mapjoin_7-8-f983875c44b290b0884a22b6be6adc8} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f rename sql/hive/src/test/resources/golden/{groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c => smb_mapjoin_7-9-84a394d962965e38593883742cc32c0d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 rename sql/hive/src/test/resources/golden/{groupby_position-7-5b32a45af11e04b46f8566bd27a28014 => smb_mapjoin_8-5-eee18fc4192a4aa92a066eb66513be93} (100%) delete mode 100644 sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e delete mode 100644 sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b delete mode 100644 sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 delete mode 100644 sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 delete mode 100644 sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 delete mode 100644 sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 delete mode 100644 sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e delete mode 100644 sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 delete mode 100644 sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b delete mode 100644 sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 delete mode 100644 sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a delete mode 100644 sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 delete mode 100644 sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 delete mode 100644 sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa delete mode 100644 sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 delete mode 100644 sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa delete mode 100644 sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc delete mode 100644 sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 delete mode 100644 sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa delete mode 100644 sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 delete mode 100644 sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 delete mode 100644 sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 delete mode 100644 sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b delete mode 100644 sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 delete mode 100644 sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 delete mode 100644 sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 delete mode 100644 sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 delete mode 100644 sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 delete mode 100644 sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 delete mode 100644 sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c delete mode 100644 sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d delete mode 100644 sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 delete mode 100644 sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 delete mode 100644 sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 delete mode 100644 sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 delete mode 100644 sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 delete mode 100644 sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c delete mode 100644 sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 delete mode 100644 sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 delete mode 100644 sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 delete mode 100644 sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 delete mode 100644 sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 delete mode 100644 sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd delete mode 100644 sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c delete mode 100644 sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd delete mode 100644 sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c delete mode 100644 sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d delete mode 100644 sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 delete mode 100644 sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f delete mode 100644 sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 delete mode 100644 sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df delete mode 100644 sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 delete mode 100644 sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 delete mode 100644 sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 delete mode 100644 sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c delete mode 100644 sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 delete mode 100644 sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc delete mode 100644 sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce delete mode 100644 sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 delete mode 100644 sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 delete mode 100644 sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 delete mode 100644 sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc delete mode 100644 sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 delete mode 100644 sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 delete mode 100644 sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 delete mode 100644 sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 delete mode 100644 sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 delete mode 100644 sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d delete mode 100644 sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 delete mode 100644 sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 delete mode 100644 sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 delete mode 100644 sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 delete mode 100644 sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c delete mode 100644 sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 delete mode 100644 sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 delete mode 100644 sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 delete mode 100644 sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 delete mode 100644 sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c delete mode 100644 sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 delete mode 100644 sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc delete mode 100644 sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 delete mode 100644 sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce delete mode 100644 sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 delete mode 100644 sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 rename sql/hive/src/test/resources/golden/{groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c => stats_aggregator_error_1-0-6f3df708fa339df236ec9375d2ad37fa} (100%) rename sql/hive/src/test/resources/golden/{exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => stats_aggregator_error_1-1-36eee5cbac5c0c3228e499805b32f6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 rename sql/hive/src/test/resources/golden/{exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee => stats_publisher_error_1-1-36eee5cbac5c0c3228e499805b32f6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 delete mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 delete mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c delete mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 rename sql/hive/src/test/resources/golden/{exim_16_part_external-1-baeaf0da490037e7ada642d23013075a => timestamp_1-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 => timestamp_1-1-d362501d0176855077e65f8faf067fa8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-18-67f274bf16de625cf4e85af0c6185cac => timestamp_1-10-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-10-343c75daac6695917608c17db8bf473e => timestamp_1-11-343c75daac6695917608c17db8bf473e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 rename sql/hive/src/test/resources/golden/{timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-12-cf19f7359a6d3456c4526b2c69f92d6a} (100%) rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d => timestamp_1-13-d242038c04dd4ee6075c7eebc0f75f17} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 => timestamp_1-14-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-15-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-16-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-17-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-27-67f274bf16de625cf4e85af0c6185cac => timestamp_1-19-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d => timestamp_1-2-1d7cf3a2512fa1876b422b79bbe05426} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 rename sql/hive/src/test/resources/golden/{timestamp_1-19-343c75daac6695917608c17db8bf473e => timestamp_1-20-343c75daac6695917608c17db8bf473e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-21-cf19f7359a6d3456c4526b2c69f92d6a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e rename sql/hive/src/test/resources/golden/{groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 => timestamp_1-22-cdb04b49b836e0244f6f0857aea7da8a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 => timestamp_1-23-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-24-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-25-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-26-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-36-67f274bf16de625cf4e85af0c6185cac => timestamp_1-28-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-28-343c75daac6695917608c17db8bf473e => timestamp_1-29-343c75daac6695917608c17db8bf473e} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c => timestamp_1-3-74f477a8b726f5193dd42ef378a793c4} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 rename sql/hive/src/test/resources/golden/{timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-30-cf19f7359a6d3456c4526b2c69f92d6a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 => timestamp_1-31-9587976bd7e6caa5b667975c14e8dd53} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 => timestamp_1-32-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-33-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-34-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-35-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-45-67f274bf16de625cf4e85af0c6185cac => timestamp_1-37-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-37-343c75daac6695917608c17db8bf473e => timestamp_1-38-343c75daac6695917608c17db8bf473e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d rename sql/hive/src/test/resources/golden/{timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-39-cf19f7359a6d3456c4526b2c69f92d6a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea => timestamp_1-4-d833b177fac3162215468dde991f71d1} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 => timestamp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-13-25f6ec69328af6cba76899194e0dd84e => timestamp_1-41-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-42-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-43-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-44-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-54-67f274bf16de625cf4e85af0c6185cac => timestamp_1-46-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-46-343c75daac6695917608c17db8bf473e => timestamp_1-47-343c75daac6695917608c17db8bf473e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 rename sql/hive/src/test/resources/golden/{timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-48-cf19f7359a6d3456c4526b2c69f92d6a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 => timestamp_1-49-7a59f9f939efc4b96f8159d00b39ed3} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-22-25f6ec69328af6cba76899194e0dd84e => timestamp_1-5-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-31-25f6ec69328af6cba76899194e0dd84e => timestamp_1-50-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-51-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-52-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-53-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-9-67f274bf16de625cf4e85af0c6185cac => timestamp_1-55-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-55-343c75daac6695917608c17db8bf473e => timestamp_1-56-343c75daac6695917608c17db8bf473e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-57-cf19f7359a6d3456c4526b2c69f92d6a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 rename sql/hive/src/test/resources/golden/{groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 => timestamp_1-58-d362501d0176855077e65f8faf067fa8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-8-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 => timestamp_2-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 rename sql/hive/src/test/resources/golden/{groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 => timestamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-18-252aebfe7882335d31bfc53a8705b7a => timestamp_2-10-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-11-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-11-240fce5f58794fa051824e8732c00c03 => timestamp_2-12-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd rename sql/hive/src/test/resources/golden/{groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 => timestamp_2-13-5f450162886ccc79af149541527f5643} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-4-25f6ec69328af6cba76899194e0dd84e => timestamp_2-14-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-15-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-16-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-17-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-27-252aebfe7882335d31bfc53a8705b7a => timestamp_2-19-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 => timestamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 rename sql/hive/src/test/resources/golden/{timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-20-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-20-240fce5f58794fa051824e8732c00c03 => timestamp_2-21-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b rename sql/hive/src/test/resources/golden/{groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 => timestamp_2-22-469fe94fb60f4b00809190c303434641} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-40-25f6ec69328af6cba76899194e0dd84e => timestamp_2-23-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-24-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-25-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-26-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-36-252aebfe7882335d31bfc53a8705b7a => timestamp_2-28-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-29-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 => timestamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 rename sql/hive/src/test/resources/golden/{timestamp_2-29-240fce5f58794fa051824e8732c00c03 => timestamp_2-30-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 rename sql/hive/src/test/resources/golden/{groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f => timestamp_2-31-8f506498acf0c99c30960a00981ef460} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-49-25f6ec69328af6cba76899194e0dd84e => timestamp_2-32-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-33-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-34-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-35-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-45-252aebfe7882335d31bfc53a8705b7a => timestamp_2-37-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-38-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-38-240fce5f58794fa051824e8732c00c03 => timestamp_2-39-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 rename sql/hive/src/test/resources/golden/{groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 => timestamp_2-4-81d6d29dcb3fd12a519426dff64411d2} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 => timestamp_2-40-972a007e54d1c09e9ac9549c19a32dbb} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 => timestamp_2-41-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-42-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-43-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-44-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-54-252aebfe7882335d31bfc53a8705b7a => timestamp_2-46-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-47-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-47-240fce5f58794fa051824e8732c00c03 => timestamp_2-48-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb rename sql/hive/src/test/resources/golden/{groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f => timestamp_2-49-650d2727b007638e0ed39b37c9498d66} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 => timestamp_2-5-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 => timestamp_2-50-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-51-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-52-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-53-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-9-252aebfe7882335d31bfc53a8705b7a => timestamp_2-55-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-56-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-56-240fce5f58794fa051824e8732c00c03 => timestamp_2-57-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 rename sql/hive/src/test/resources/golden/{groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef => timestamp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-6-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-7-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 rename sql/hive/src/test/resources/golden/{exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee => timestamp_3-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 => timestamp_3-1-165256158e3db1ce19c3c9db3c8011d2} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b rename sql/hive/src/test/resources/golden/{timestamp_3-9-ffc79abb874323e165963aa39f460a9b => timestamp_3-10-ffc79abb874323e165963aa39f460a9b} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 => timestamp_3-11-7b1ec929239ee305ea9da46ebb990c67} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 rename sql/hive/src/test/resources/golden/{timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a => timestamp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a} (100%) create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 rename sql/hive/src/test/resources/golden/{groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 => timestamp_3-14-165256158e3db1ce19c3c9db3c8011d2} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 rename sql/hive/src/test/resources/golden/{groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d => timestamp_3-2-81edf5107270547641586aa02b4e7d9b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a rename sql/hive/src/test/resources/golden/{groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b => timestamp_3-3-7a012a0d98729da25b5ac374855dcee4} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad => timestamp_3-4-86514381187b246a5685577c1968c559} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 => timestamp_3-5-935d0d2492beab99bbbba26ba62a1db4} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 => timestamp_3-6-8fe348d5d9b9903a26eda32d308b8e41} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 => timestamp_3-7-6be5fe01c502cd24db32a3781c97a703} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add => timestamp_3-8-6066ba0451cd0fcfac4bea6376e72add} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-8-22e03daa775eab145d39ec0730953f7e => timestamp_3-9-22e03daa775eab145d39ec0730953f7e} (100%) rename sql/hive/src/test/resources/golden/{exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a => timestamp_comparison-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 => timestamp_comparison-1-4b68f7ad0f8cf337e42bf16a45e15818} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d => timestamp_comparison-2-60557e7bd2822c89fa8b076a9d0520fc} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc => timestamp_comparison-3-f96a9d88327951bd93f672dc2463ecd4} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-14-773801b833cf72d35016916b786275b5 => timestamp_comparison-4-13e17ed811165196416f777cbc162592} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 => timestamp_comparison-5-4fa8a36edbefde4427c2ab2cf30e6399} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 => timestamp_comparison-6-7e4fb6e8ba01df422e4c67e06a0c8453} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 => timestamp_comparison-7-8c8e73673a950f6b3d960b08fcea076f} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 => timestamp_comparison-8-510c0a2a57dc5df8588bd13c4152f8bc} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 => timestamp_comparison-9-659d5b1ae8200f13f265270e52a3dd65} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 => timestamp_lazy-2-bb5a4a13274290029bd07d95c2f92563} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 rename sql/hive/src/test/resources/golden/{groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c => timestamp_null-2-51762cf5079877abf7d81127738f4e5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 rename sql/hive/src/test/resources/golden/{exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 => timestamp_udf-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d rename sql/hive/src/test/resources/golden/{groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a => timestamp_udf-1-79914c5347620c6e62a8e0b9a95984af} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-18-cb033ecad964a2623bc633ac1d3f752a => timestamp_udf-10-287614364eaa3fb82aad08c6b62cc938} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-10-dbc23736a61d9482d13cacada02a7a09 => timestamp_udf-11-dbc23736a61d9482d13cacada02a7a09} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-11-442cf850a0cc1f1dcfdeaeffbffb2c35 => timestamp_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-12-51959036fd4ac4f1e24f4e06eb9b0b6 => timestamp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-13-6ab3f356deaf807e8accc37e1f4849a => timestamp_udf-14-6ab3f356deaf807e8accc37e1f4849a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-14-c745a1016461403526d44928a269c1de => timestamp_udf-15-c745a1016461403526d44928a269c1de} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-15-7ab76c4458c7f78038c8b1df0fdeafbe => timestamp_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-16-b36e87e17ca24d82072220bff559c718 => timestamp_udf-17-b36e87e17ca24d82072220bff559c718} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-17-dad44d2d4a421286e9da080271bd2639 => timestamp_udf-18-dad44d2d4a421286e9da080271bd2639} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-19-79914c5347620c6e62a8e0b9a95984af rename sql/hive/src/test/resources/golden/{timestamp_udf-9-287614364eaa3fb82aad08c6b62cc938 => timestamp_udf-19-cb033ecad964a2623bc633ac1d3f752a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 => timestamp_udf-2-59fc1842a23369235d42ed040d45fb3d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-20-59fc1842a23369235d42ed040d45fb3d rename sql/hive/src/test/resources/golden/{groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 => timestamp_udf-20-79914c5347620c6e62a8e0b9a95984af} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa => timestamp_udf-21-59fc1842a23369235d42ed040d45fb3d} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 => timestamp_udf-3-9039f474f9a96e9f15ace528faeed923} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-4-80ce02ec84ee8abcb046367ca37279cc rename sql/hive/src/test/resources/golden/{groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b => timestamp_udf-4-b0fd4ca3b22eb732a32772399331352f} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 => timestamp_udf-5-66868a2b075de978784011e9955483d} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-5-1124399033bcadf3874fb48f593392d => timestamp_udf-6-1124399033bcadf3874fb48f593392d} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-6-5810193ce35d38c23f4fc4b4979d60a4 => timestamp_udf-7-5810193ce35d38c23f4fc4b4979d60a4} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-7-250e640a6a818f989f3f3280b00f64f9 => timestamp_udf-8-250e640a6a818f989f3f3280b00f64f9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-8-975df43df015d86422965af456f87a94 => timestamp_udf-9-975df43df015d86422965af456f87a94} (100%) delete mode 100644 sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 delete mode 100644 sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 delete mode 100644 sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 delete mode 100644 sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 delete mode 100644 sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 delete mode 100644 sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 delete mode 100644 sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f rename sql/hive/src/test/resources/golden/{exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee => type_cast_1-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d rename sql/hive/src/test/resources/golden/{groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 => type_cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d} (100%) create mode 100644 sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 rename sql/hive/src/test/resources/golden/{exim_18_part_external-1-baeaf0da490037e7ada642d23013075a => type_widening-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 rename sql/hive/src/test/resources/golden/{groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 => type_widening-1-630ac2c7e7dea4837384ccd572209229} (100%) delete mode 100644 sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 delete mode 100644 sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 rename sql/hive/src/test/resources/golden/{exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 => type_widening-2-cfbdf2b6fca84c6e23d4e691d2221bd6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 rename sql/hive/src/test/resources/golden/{groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c => type_widening-3-a0ba6952d9bf830d1d1ea7aebd3784a2} (100%) create mode 100644 sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee => udaf_collect_set-12-1d351f7e821fcaf66c6f7503e42fb291} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a => udaf_collect_set-13-a00d1791b7fa7ac5a0505d95c3d12257} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => udaf_collect_set-15-863233ccd616401efb4bf83c4b9e3a52} (100%) rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee => udaf_collect_set-16-a00d1791b7fa7ac5a0505d95c3d12257} (100%) create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a => udaf_collect_set-4-1d351f7e821fcaf66c6f7503e42fb291} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => udaf_collect_set-5-a7dc16cb82c595b18d4258a38a304b1e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee => udaf_collect_set-8-863233ccd616401efb4bf83c4b9e3a52} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a => udaf_collect_set-9-a7dc16cb82c595b18d4258a38a304b1e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 rename sql/hive/src/test/resources/golden/{groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 => udaf_corr-2-e886f45c8f085596ffd420f89cdc2909} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 rename sql/hive/src/test/resources/golden/{groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 => udaf_covar_pop-2-e886f45c8f085596ffd420f89cdc2909} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 rename sql/hive/src/test/resources/golden/{groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 => udaf_covar_samp-2-e886f45c8f085596ffd420f89cdc2909} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 delete mode 100644 sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 delete mode 100644 sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 delete mode 100644 sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da delete mode 100644 sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f delete mode 100644 sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c delete mode 100644 sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 delete mode 100644 sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e delete mode 100644 sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => udf_E-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b => udf_E-1-cad0779d18f326c8e453bf2b5fe43596} (100%) rename sql/hive/src/test/resources/golden/{udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c => udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02} (100%) rename sql/hive/src/test/resources/golden/{udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a => udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f rename sql/hive/src/test/resources/golden/{udf_E-3-e8924af3bf99d2e01546a965303ffd09 => udf_E-4-e8924af3bf99d2e01546a965303ffd09} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 => udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4} (100%) rename sql/hive/src/test/resources/golden/{udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c => udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02} (100%) rename sql/hive/src/test/resources/golden/{udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a => udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a} (100%) rename sql/hive/src/test/resources/golden/{udf_E-7-e8924af3bf99d2e01546a965303ffd09 => udf_E-8-e8924af3bf99d2e01546a965303ffd09} (100%) rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee => udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f rename sql/hive/src/test/resources/golden/{groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 => udf_PI-1-13fd9345fd15b654d18b2707e5274b2b} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 => udf_PI-2-97a12f6967726e425469ecfa70177ff0} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-2-9c1476a2eab7455594e97b338ee3c188 => udf_PI-3-9c1476a2eab7455594e97b338ee3c188} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 => udf_PI-4-890f3c276eff2c459d8dc79d5a71c866} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f rename sql/hive/src/test/resources/golden/{groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 => udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 => udf_PI-6-97a12f6967726e425469ecfa70177ff0} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-6-9c1476a2eab7455594e97b338ee3c188 => udf_PI-7-9c1476a2eab7455594e97b338ee3c188} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 => udf_PI-8-890f3c276eff2c459d8dc79d5a71c866} (100%) rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a => udf_abs-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 rename sql/hive/src/test/resources/golden/{groupby_sort_1-67-b4fec0996399be2239961594897d6715 => udf_abs-3-50cb3c23902cd29e0dbff188c71062e5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c create mode 100644 sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 rename sql/hive/src/test/resources/golden/{groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 => udf_abs-5-5cd4c198e0de884ad436864b95fece6c} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e => udf_acos-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b create mode 100644 sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f rename sql/hive/src/test/resources/golden/{decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 => udf_acos-3-661a0a85283df2a5c1567d60850e362b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 create mode 100644 sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a create mode 100644 sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e create mode 100644 sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee => udf_array-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee => udf_array-1-2e9c2a7d9325bd9a53cf9f181b6333ee} (100%) rename sql/hive/src/test/resources/golden/{udf_array-1-570741914bb78300b0233e5f38d7f08a => udf_array-2-570741914bb78300b0233e5f38d7f08a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 rename sql/hive/src/test/resources/golden/{groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc => udf_array-3-47818d42e5e7667d8754c3f9a4b8053a} (100%) rename sql/hive/src/test/resources/golden/{udf_array-3-a5d12c41277fb158e09281169c905122 => udf_array-4-51410e4d4d679fe5a8dd7a860f4efc47} (100%) rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a => udf_array_contains-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a create mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a delete mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c delete mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 create mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c rename sql/hive/src/test/resources/golden/{udf_to_boolean-3-266b9601a9154438ab95550fcd36494c => udf_array_contains-3-6b700cb69af298aef45b96bf5ac862d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 rename sql/hive/src/test/resources/golden/{udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d => udf_array_contains-4-bdb5a7e0ab81172a438145a1c406b1e8} (100%) rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 => udf_ascii-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c delete mode 100644 sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 delete mode 100644 sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 delete mode 100644 sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 rename sql/hive/src/test/resources/golden/{groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e => udf_ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee => udf_asin-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd create mode 100644 sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f rename sql/hive/src/test/resources/golden/{decimal_precision-12-18906f5c6413065621430e3fe33c7e9e => udf_asin-3-4b7ee6310a49ebf784a4a712748348ac} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a => udf_atan-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b create mode 100644 sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 rename sql/hive/src/test/resources/golden/{decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e => udf_atan-3-e6f97a834028a67e6c3033949f98fbf8} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b create mode 100644 sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b create mode 100644 sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 rename sql/hive/src/test/resources/golden/{metadataonly1-2-9eadfd16be30c653625fce7b74048d9d => udf_atan-7-e6f97a834028a67e6c3033949f98fbf8} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 => udf_bin-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b delete mode 100644 sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 create mode 100644 sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b delete mode 100644 sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b create mode 100644 sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 create mode 100644 sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab delete mode 100644 sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 create mode 100644 sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee => udf_bitmap_and-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 rename sql/hive/src/test/resources/golden/{groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 => udf_bitmap_and-3-d550d017f9eb9176593719ea34eaae9b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 rename sql/hive/src/test/resources/golden/{groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 => udf_bitmap_and-4-a486db1f5a06f9623a0e5abd7737b0c6} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 => udf_bitmap_and-5-549dbeb1293c4c49ae08bf08acdbdf23} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 rename sql/hive/src/test/resources/golden/{groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 => udf_bitmap_and-7-d550d017f9eb9176593719ea34eaae9b} (100%) rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a => udf_bitmap_empty-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df rename sql/hive/src/test/resources/golden/{udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad => udf_bitmap_empty-1-a174269b5d1757398ab7f89cf1c97bfa} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 => udf_bitmap_empty-2-f5d9880a3278b5632b356bbe6ecd90d3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 => udf_bitmap_or-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 rename sql/hive/src/test/resources/golden/{groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d => udf_bitmap_or-3-d550d017f9eb9176593719ea34eaae9b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 rename sql/hive/src/test/resources/golden/{groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 => udf_bitmap_or-4-a486db1f5a06f9623a0e5abd7737b0c6} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 => udf_bitmap_or-5-549dbeb1293c4c49ae08bf08acdbdf23} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 rename sql/hive/src/test/resources/golden/{groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 => udf_bitmap_or-7-d550d017f9eb9176593719ea34eaae9b} (100%) rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 => udf_case-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 delete mode 100644 sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 create mode 100644 sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 create mode 100644 sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 delete mode 100644 sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f rename sql/hive/src/test/resources/golden/{groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 => udf_case-3-48789112d79aeb450d9f49184fc20e1c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c delete mode 100644 sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 create mode 100644 sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 create mode 100644 sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb create mode 100644 sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc delete mode 100644 sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c delete mode 100644 sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 delete mode 100644 sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 delete mode 100644 sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 => udf_concat-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 => udf_concat_ws-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 rename sql/hive/src/test/resources/golden/{groupby_sort_11-4-475d50465b23adfb70e67122425ede9e => udf_concat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f rename sql/hive/src/test/resources/golden/{groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 => udf_concat_ws-4-b13a1f7f63d2a54efa331c82bd635d63} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 => udf_concat_ws-5-a507af4996b13433b0ae100fcb32358f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c rename sql/hive/src/test/resources/golden/{groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c => udf_concat_ws-7-97071809ba7701b78e3729996f14b591} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 => udf_conv-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab create mode 100644 sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f create mode 100644 sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a create mode 100644 sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f create mode 100644 sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb create mode 100644 sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 delete mode 100644 sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 => udf_cos-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 delete mode 100644 sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 rename sql/hive/src/test/resources/golden/{udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 => udf_cos-3-166acc86afd6ececfe43800e38f106c9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 => udf_degrees-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae rename sql/hive/src/test/resources/golden/{groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_degrees-1-f24ce67606944e23a4adc79f91cf0c17} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-1-aabc6065a03b7da809376cc127af47d7 => udf_degrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b => udf_degrees-3-2fd3a55901fe765f8f154531a7f5fd6b} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 => udf_degrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae rename sql/hive/src/test/resources/golden/{groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_degrees-5-3a6468b02be2605c91b31987e76fb9a8} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-5-aabc6065a03b7da809376cc127af47d7 => udf_degrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b => udf_degrees-7-2fd3a55901fe765f8f154531a7f5fd6b} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 => udf_degrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db rename sql/hive/src/test/resources/golden/{groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e => udf_div-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db delete mode 100644 sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 delete mode 100644 sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b create mode 100644 sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 rename sql/hive/src/test/resources/golden/{custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 => udf_div-3-5111340caad64e36370d9d4bc4db5f27} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 delete mode 100644 sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 delete mode 100644 sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 rename sql/hive/src/test/resources/golden/{groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 => udf_elt-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_elt-0-b46b060da76d1772db998c26a62a608f => udf_elt-1-b46b060da76d1772db998c26a62a608f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a rename sql/hive/src/test/resources/golden/{udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 => udf_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_elt-3-c2554fac72a2a51bb33faae16aec3507} (100%) rename sql/hive/src/test/resources/golden/{udf_elt-3-f3be980cf4fa166f299c6ec79e981814 => udf_elt-4-533ad9c703c320a6556c09dd5f9ac351} (100%) rename sql/hive/src/test/resources/golden/{groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 => udf_equal-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 => udf_equal-1-36b6cdf7c5f68c91155569b1622f5876} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-1-2422b50b96502dde8b661acdfebd8892 => udf_equal-2-2422b50b96502dde8b661acdfebd8892} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 => udf_equal-3-e0faab0f5e736c24bcc5503aeac55053} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-3-39d8d6f197803de927f0af5409ec2f33 => udf_equal-4-39d8d6f197803de927f0af5409ec2f33} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 => udf_equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 => udf_equal-6-878650cf21e9360a07d204c8ffb0cde7} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e => udf_equal-7-1635ef051fecdfc7891d9f5a9a3a545e} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-7-78f1b96c199e307714fa1b804e5bae27 => udf_equal-8-276101b04b10b7cd6d59061a8cbf42d2} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 => udf_field-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 delete mode 100644 sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f delete mode 100644 sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 rename sql/hive/src/test/resources/golden/{groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_field-10-ca9db7e6bb687606bc273d1f6c191035} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 create mode 100644 sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f delete mode 100644 sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b rename sql/hive/src/test/resources/golden/{div-0-5e7fc5719c7265bc1d6af52005ebad03 => udf_field-3-5960d42257b272f6ba043406229bbf26} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 create mode 100644 sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 delete mode 100644 sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 delete mode 100644 sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 create mode 100644 sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b rename sql/hive/src/test/resources/golden/{groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_field-6-2d7c5cbe891c4a9dda34f9842f8e0828} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 rename sql/hive/src/test/resources/golden/{groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_field-7-3fd8b0c333acdf28c676315b03e2e10} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef delete mode 100644 sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 create mode 100644 sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef rename sql/hive/src/test/resources/golden/{groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 => udf_field-9-66dc6c81db0ac9b2075783b0d8976083} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 => udf_find_in_set-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c rename sql/hive/src/test/resources/golden/{udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 => udf_find_in_set-10-df21f44247d7275a292520c1605c4aab} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a rename sql/hive/src/test/resources/golden/{decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e => udf_find_in_set-11-91f8c37820f31d0d1b16029a59a185ad} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 rename sql/hive/src/test/resources/golden/{decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 => udf_find_in_set-12-692f41c998bbc6bec0f374eae4356739} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 rename sql/hive/src/test/resources/golden/{decimal_2-25-14face5c7104382196e65741a199c36 => udf_find_in_set-14-8e410ecfad2d408ad7d2554ccd3a6621} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 => udf_find_in_set-15-c50e6ff95c05bfa854b33b03db858cd9} (100%) rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 => udf_find_in_set-16-8e17f41ae6e8b1075af4790a8fd88e13} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 => udf_find_in_set-17-fe61f992f2d971d006155bdec3143803} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e rename sql/hive/src/test/resources/golden/{groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb => udf_find_in_set-3-b3823bdc04a7f98951b55c3e30d2a772} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b rename sql/hive/src/test/resources/golden/{groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 => udf_find_in_set-5-6f25b5bba89e1fcae171f5d595acc4ee} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 rename sql/hive/src/test/resources/golden/{decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 => udf_find_in_set-7-730d5e95ef748ad946eceefbcd633826} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 => udf_find_in_set-8-ea11724531f191940e455d13878a0e69} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 => udf_find_in_set-9-81f9999ed1b063ce7f17d89bd0162777} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 rename sql/hive/src/test/resources/golden/{groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 => udf_format_number-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 rename sql/hive/src/test/resources/golden/{groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb => udf_format_number-1-e86d559aeb84a4cc017a103182c22bfb} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 rename sql/hive/src/test/resources/golden/{groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb => udf_format_number-4-7969ffc4e80f7214a8eead8e1084368a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 rename sql/hive/src/test/resources/golden/{groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 => udf_greaterthan-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 rename sql/hive/src/test/resources/golden/{groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 => udf_greaterthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 rename sql/hive/src/test/resources/golden/{groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 => udf_hash-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_hash-0-b9e3a3986320d275982797140edfccf4 => udf_hash-1-b9e3a3986320d275982797140edfccf4} (100%) rename sql/hive/src/test/resources/golden/{udf_hash-1-a18646b51501d0b1beb967dc79afbd1a => udf_hash-2-a18646b51501d0b1beb967dc79afbd1a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 => udf_hash-3-2646a87ce26c383a9dafea9b56281ab7} (100%) rename sql/hive/src/test/resources/golden/{udf_hash-3-23991312391d518aacf3d4469c816eae => udf_hash-4-d1368c2e3cd113e46202156b44811987} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 => udf_hex-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 create mode 100644 sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 create mode 100644 sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c create mode 100644 sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf create mode 100644 sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea create mode 100644 sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a rename sql/hive/src/test/resources/golden/{input16_cc-0-43d53504df013e6b35f81811138a167a => udf_if-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_if-0-b7ffa85b5785cccef2af1b285348cc2c => udf_if-1-b7ffa85b5785cccef2af1b285348cc2c} (100%) rename sql/hive/src/test/resources/golden/{udf_if-1-30cf7f51f92b5684e556deff3032d49a => udf_if-2-30cf7f51f92b5684e556deff3032d49a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 => udf_if-3-59e90bb74481aaf35480076806daf365} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca rename sql/hive/src/test/resources/golden/{udf_if-3-20206f17367ff284d67044abd745ce9f => udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c => udf_if-5-841a8068d35a42179d3654e1a2848c43} (100%) rename sql/hive/src/test/resources/golden/{udf_if-5-a7db13aec05c97792f9331d63709d8cc => udf_if-6-508f9140dd33931c7b9ad336dceb32cf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 delete mode 100644 sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 delete mode 100644 sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 delete mode 100644 sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 delete mode 100644 sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 rename sql/hive/src/test/resources/golden/{input31-0-823920925ca9c8a2ca9016f52c0f4ee => udf_index-0-e91e3e5a22029b9b979ccbbef97add66} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 create mode 100644 sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 create mode 100644 sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 rename sql/hive/src/test/resources/golden/{input31-1-c21dba410fb07a098f93430a9d21df79 => udf_instr-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 delete mode 100644 sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 delete mode 100644 sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 create mode 100644 sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 delete mode 100644 sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 => udf_instr-3-c40fbd09410b11388ce7a6e9bea5846f} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b delete mode 100644 sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 delete mode 100644 sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a delete mode 100644 sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 delete mode 100644 sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd delete mode 100644 sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b rename sql/hive/src/test/resources/golden/{input32-0-823920925ca9c8a2ca9016f52c0f4ee => udf_length-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b delete mode 100644 sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea => udf_length-10-de456a5765db4a06110d9483985aa4a6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 create mode 100644 sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 delete mode 100644 sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 create mode 100644 sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 => udf_length-3-af46cb6887618240836eaf5be8afbba6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d delete mode 100644 sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 => udf_length-4-dcd6404afce1103d5054527e6c216d6d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 => udf_length-5-ba49ba4e6505c74bc33b5003f3930c43} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a create mode 100644 sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 => udf_length-7-460dffb0f1ab0ac0ebc4fd545809aa9a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f delete mode 100644 sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 => udf_length-8-8f28e6c488df47d89dca670f04a7563f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 => udf_length-9-e41b220da98996f997b26ba7ef457a84} (100%) rename sql/hive/src/test/resources/golden/{input32-1-c21dba410fb07a098f93430a9d21df79 => udf_lessthan-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 rename sql/hive/src/test/resources/golden/{input32-2-1ba7748b3d2f8908c2e81771ab229316 => udf_lessthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 rename sql/hive/src/test/resources/golden/{join_view-4-763ab5853bff619e6525c01e46b2a923 => udf_like-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_like-0-e0ba9a953e50554bdcbc55585cffde09 => udf_like-1-e0ba9a953e50554bdcbc55585cffde09} (100%) rename sql/hive/src/test/resources/golden/{udf_like-1-9781f89d352c506e972ad2a1d58ec03a => udf_like-2-9781f89d352c506e972ad2a1d58ec03a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 => udf_like-3-dbc46cb33f0dd356af03006d9492f8b7} (100%) rename sql/hive/src/test/resources/golden/{udf_like-3-bef03784eab9d5e8404fd24960dea4fc => udf_like-4-bef03784eab9d5e8404fd24960dea4fc} (100%) rename sql/hive/src/test/resources/golden/{udf_like-4-af5fe5d5d176f751747bf14055d00a12 => udf_like-5-47bfd4d65090dab890b467ae06cf3bd5} (100%) rename sql/hive/src/test/resources/golden/{load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 => udf_locate-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e create mode 100644 sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e delete mode 100644 sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 delete mode 100644 sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c create mode 100644 sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 delete mode 100644 sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 => udf_locate-3-ce4a131f99dc9befa926027380b38dbb} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b delete mode 100644 sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 rename sql/hive/src/test/resources/golden/{load_dyn_part15-1-16367c381d4b189b3640c92511244bfe => udf_lpad-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef delete mode 100644 sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f => udf_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 rename sql/hive/src/test/resources/golden/{load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e => udf_map-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e => udf_map-1-9feb9f29115f94b3bb4f6a36160bd17e} (100%) rename sql/hive/src/test/resources/golden/{udf_map-1-1f8cd98df9bf7b2528506551fef87dcf => udf_map-2-1f8cd98df9bf7b2528506551fef87dcf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 => udf_map-3-be7b52baa973b8b59b7ca63fea19aa99} (100%) rename sql/hive/src/test/resources/golden/{udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 => udf_map-4-60cb9c30285f7a9f99377ccbd143eb06} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e delete mode 100644 sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f delete mode 100644 sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 rename sql/hive/src/test/resources/golden/{load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e => udf_negative-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf rename sql/hive/src/test/resources/golden/{udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 => udf_negative-10-1cd28efecc0d26f463221195f5e39956} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c create mode 100644 sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 create mode 100644 sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c rename sql/hive/src/test/resources/golden/{udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a => udf_negative-5-771e76b0acd8ddb128781da7819d0e47} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 rename sql/hive/src/test/resources/golden/{udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d => udf_negative-6-f62c4a097c592871d896a7dc47c42f61} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b rename sql/hive/src/test/resources/golden/{udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 => udf_negative-7-f838053f5ca5c8746dc299473dff0490} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 rename sql/hive/src/test/resources/golden/{udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da => udf_negative-8-f4f23aa6f634913d194a69261af8f3f6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa rename sql/hive/src/test/resources/golden/{udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 => udf_negative-9-f6a78fa3ea0f519d0e4abc5be7a960e5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e rename sql/hive/src/test/resources/golden/{load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e => udf_notequal-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 => udf_notequal-5-eb04e5ee00967799c913e8a5b424a332} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f => udf_notequal-7-46a6514f2d7e6a097035ec1559df0096} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd rename sql/hive/src/test/resources/golden/{loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 => udf_notop-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 create mode 100644 sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 rename sql/hive/src/test/resources/golden/{lock3-3-16367c381d4b189b3640c92511244bfe => udf_nvl-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 delete mode 100644 sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 delete mode 100644 sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef => udf_nvl-3-47199a1c23cb1cc6827c601bb66513d3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 delete mode 100644 sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 delete mode 100644 sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a delete mode 100644 sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 rename sql/hive/src/test/resources/golden/{lock3-4-a4fb8359a2179ec70777aad6366071b7 => udf_radians-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 => udf_radians-1-58b73fc96927d447d1225f021eaa378} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 => udf_radians-6-70c9e7199b5898e2c3a4943ec58da113} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca rename sql/hive/src/test/resources/golden/{lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 => udf_regexp-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_regexp-0-19917611f74aedc0922560f7f2595948 => udf_regexp-1-19917611f74aedc0922560f7f2595948} (100%) rename sql/hive/src/test/resources/golden/{udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 => udf_regexp-2-f7f0527cd47612d7f256edd5f8963800} (100%) rename sql/hive/src/test/resources/golden/{udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 => udf_regexp-3-59aff54bae544ee620141e4e629f167a} (100%) rename sql/hive/src/test/resources/golden/{lock4-4-16367c381d4b189b3640c92511244bfe => udf_repeat-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b delete mode 100644 sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b delete mode 100644 sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d => udf_repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 rename sql/hive/src/test/resources/golden/{lock4-5-a4fb8359a2179ec70777aad6366071b7 => udf_round-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 create mode 100644 sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c create mode 100644 sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de create mode 100644 sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 create mode 100644 sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d delete mode 100644 sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a create mode 100644 sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b create mode 100644 sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b rename sql/hive/src/test/resources/golden/{merge4-0-b12e5c70d6d29757471b900b6160fa8a => udf_round_3-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad rename sql/hive/src/test/resources/golden/{merge4-1-593999fae618b6b38322bc9ae4e0c027 => udf_rpad-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c delete mode 100644 sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c delete mode 100644 sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b => udf_rpad-3-66acb969c28a8e376782ccd0d442b450} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b rename sql/hive/src/test/resources/golden/{merge4-2-43d53504df013e6b35f81811138a167a => udf_second-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 delete mode 100644 sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 create mode 100644 sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 create mode 100644 sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 delete mode 100644 sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f delete mode 100644 sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 => udf_second-3-d678372e3837a16be245d2e33482f17f} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e rename sql/hive/src/test/resources/golden/{merge4-3-a4fb8359a2179ec70777aad6366071b7 => udf_sign-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c => udf_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 rename sql/hive/src/test/resources/golden/{udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a => udf_sign-10-9a5326b0bf612fed4ce0b04770bebc16} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a create mode 100644 sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa create mode 100644 sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef create mode 100644 sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d create mode 100644 sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 rename sql/hive/src/test/resources/golden/{udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c => udf_sign-4-9a5326b0bf612fed4ce0b04770bebc16} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a create mode 100644 sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e create mode 100644 sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a => udf_sign-7-ed2aaa1a416c0cccc04de970424e1860} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef create mode 100644 sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d create mode 100644 sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 delete mode 100644 sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee rename sql/hive/src/test/resources/golden/{merge4-4-16367c381d4b189b3640c92511244bfe => udf_sin-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 create mode 100644 sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee create mode 100644 sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 delete mode 100644 sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab rename sql/hive/src/test/resources/golden/{udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 => udf_sin-3-1d3a615e3aa252a317daa601811820b1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 create mode 100644 sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a delete mode 100644 sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 delete mode 100644 sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 delete mode 100644 sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a rename sql/hive/src/test/resources/golden/{metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 => udf_space-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 create mode 100644 sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 delete mode 100644 sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 delete mode 100644 sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 create mode 100644 sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 delete mode 100644 sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 => udf_space-3-a1b9dad63547f7ba73a5230d650983b0} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 create mode 100644 sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 create mode 100644 sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 delete mode 100644 sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 delete mode 100644 sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 delete mode 100644 sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 delete mode 100644 sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d rename sql/hive/src/test/resources/golden/{newline-1-a19a19272149c732977c37e043910505 => udf_tan-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 rename sql/hive/src/test/resources/golden/{udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 => udf_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 rename sql/hive/src/test/resources/golden/{udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 => udf_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c delete mode 100644 sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad delete mode 100644 sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 rename sql/hive/src/test/resources/golden/{orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 => udf_to_byte-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 rename sql/hive/src/test/resources/golden/{udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 => udf_to_byte-1-94eb069fb446b7758f7e06386486bec9} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 => udf_to_byte-2-233102b562824cf38010868478e91e1} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc => udf_to_byte-3-5dc0e4c21764683d98700860d2c8ab31} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 rename sql/hive/src/test/resources/golden/{udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b => udf_to_byte-4-dafb27507b4d30fd2231680f9ea80c82} (100%) rename sql/hive/src/test/resources/golden/{exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 => udf_to_byte-5-eb6600cd2260e8e75253e7844c0d7dc2} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 => udf_to_byte-6-489cd2d26b9efde2cdbff19254289371} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 => udf_to_byte-7-1eaba393f93af1763dd761172fb78d52} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 => udf_to_byte-8-8fe36cf8fba87514744a89fe50414f79} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 => udf_to_byte-9-322163c32973ccc3a5168463db7a8589} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c rename sql/hive/src/test/resources/golden/{orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb => udf_to_long-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 rename sql/hive/src/test/resources/golden/{udf_to_float-0-7646eca02448547eedf84a81bf42be89 => udf_to_long-1-8aeb3ba62d4a0cecfff363741e8042f6} (100%) rename sql/hive/src/test/resources/golden/{input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 => udf_to_long-2-7e8fa1ae8d00a121ec14941a48d24947} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 => udf_to_long-3-6e5936fba8e7486beb9ab998548bbe9b} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-3-cb3318ba365833316645e1b9890f4613 => udf_to_long-4-8c284b082a256abf0426d4f6f1971703} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 => udf_to_long-5-6bb29b93f6b0f1427ba93efb4e78810a} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-5-a0d382d243a226f8d4381970b6831c3d => udf_to_long-6-290b5a4ce01563482e81b3b532ebf9db} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-6-47ab11eae68329cc80232fc4089479f0 => udf_to_long-7-da20f84586dac3e50ee9d5b9078f44db} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-7-b87c0cabb166f33984cc1b191694918e => udf_to_long-8-90f068b4b6275bdd1c4c431fb7fa90e2} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-8-52a24d7040db321a842f9201d245ee9 => udf_to_long-9-cc8b79539085fe0e00f672b562c51cd0} (100%) rename sql/hive/src/test/resources/golden/{orc_create-32-16367c381d4b189b3640c92511244bfe => udf_to_short-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a delete mode 100644 sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b rename sql/hive/src/test/resources/golden/{udf_to_long-0-ebe447e8fb03de8844951250fe50320a => udf_to_short-1-94f15fe043839493107058a06a210cf7} (100%) rename sql/hive/src/test/resources/golden/{loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 => udf_to_short-2-981e4cb6654fde7eb4634c7ad72f8570} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 => udf_to_short-3-b259ee30ecf279bb4ad12d1515ca2767} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce => udf_to_short-4-40ffb132d5641645e2b8043dc056fb0} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 => udf_to_short-5-5d6c46b0154d1073c035a79dbf612479} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 => udf_to_short-6-6561b41835a21f973cbbc2dd80eef87f} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 => udf_to_short-7-9f83813005b639a23901ca6ff87ff473} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-7-827f3181c216fd2e990637c9a091bf0d => udf_to_short-8-885656e165feb3a674cf636dbf08716c} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 => udf_to_short-9-750382fa1a1b3ed5dca0d549d3a68996} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b rename sql/hive/src/test/resources/golden/{orc_create-34-3b03210f94ec40db9ab02620645014d1 => udf_translate-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae create mode 100644 sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf create mode 100644 sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 => udf_translate-3-42aba80bf1913dd7c64545831f476c58} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa => udf_translate-4-20904c8be8fed5cbd2d66ead6248a60a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 => udf_translate-5-5d4abaf86254bacaa545c769bd7e50ba} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad => udf_translate-6-f2637240d227f9732d3db76f2e9d3a59} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 delete mode 100644 sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 delete mode 100644 sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 delete mode 100644 sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 delete mode 100644 sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 delete mode 100644 sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c delete mode 100644 sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 delete mode 100644 sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 delete mode 100644 sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 rename sql/hive/src/test/resources/golden/{parallel_orderby-10-767848077fc2929ccedfd28e50564b19 => udf_unix_timestamp-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb => udf_unix_timestamp-1-d555c8cd733572bfa8cd3362da9480cb} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 => udf_unix_timestamp-2-8a9dbadae706047715cf5f903ff4a724} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 => udf_unix_timestamp-3-28c40e51e55bed62693e626efda5d9c5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 => udf_unix_timestamp-4-6059ff48788d0fb8317fd331172ecea9} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 => udf_unix_timestamp-5-b2e42ebb75cecf09961d36587797f6d0} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 => udf_unix_timestamp-6-31243f5cb64356425b9f95ba011ac9d6} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 => udf_unix_timestamp-7-9b0f20bde1aaf9102b67a5498b167f31} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d => udf_unix_timestamp-8-47f433ff6ccce4c666440cc1a228a96d} (100%) rename sql/hive/src/test/resources/golden/{parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 => udf_weekofyear-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 delete mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 delete mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 rename sql/hive/src/test/resources/golden/{parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 => udf_when-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 delete mode 100644 sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 create mode 100644 sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 create mode 100644 sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 delete mode 100644 sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 => udf_when-3-734890c41528b9d918db66b0582228a4} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f create mode 100644 sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 create mode 100644 sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 rename sql/hive/src/test/resources/golden/{parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 => udf_xpath-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 rename sql/hive/src/test/resources/golden/{partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 => udf_xpath_boolean-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 rename sql/hive/src/test/resources/golden/{udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f => udf_xpath_boolean-3-9b97a08303a7a89e7575687f6d7ba435} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 rename sql/hive/src/test/resources/golden/{udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df => udf_xpath_boolean-4-284ce7d6fc6850ca853111145784286b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 rename sql/hive/src/test/resources/golden/{udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a => udf_xpath_boolean-5-95a0dfa897ba9050ad751a78aeb72f3d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f rename sql/hive/src/test/resources/golden/{udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 => udf_xpath_boolean-6-796c717c9d52d3efa374d12fe65259e6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 rename sql/hive/src/test/resources/golden/{udf_to_boolean-8-37229f303635a030f6cab20e0381f51f => udf_xpath_boolean-8-62a1af222d4e12c59cef71b979c6e58} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 rename sql/hive/src/test/resources/golden/{partition_decode_name-3-16367c381d4b189b3640c92511244bfe => udf_xpath_double-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 rename sql/hive/src/test/resources/golden/{partition_decode_name-4-e90740a9a52c37a964ee204379f01412 => udf_xpath_float-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a rename sql/hive/src/test/resources/golden/{partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 => udf_xpath_int-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba rename sql/hive/src/test/resources/golden/{partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 => udf_xpath_int-3-a04ed6832ab542d6ee5903039511a826} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 rename sql/hive/src/test/resources/golden/{partition_special_char-3-16367c381d4b189b3640c92511244bfe => udf_xpath_int-4-bf5a4dbb7a98abc91111a3798b56809f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a rename sql/hive/src/test/resources/golden/{partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf => udf_xpath_int-6-ac509f06f01c02924adef220404fc515} (100%) rename sql/hive/src/test/resources/golden/{metadataonly1-5-9eadfd16be30c653625fce7b74048d9d => udf_xpath_int-7-87ff12d650afb8f511d906778450fae7} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 rename sql/hive/src/test/resources/golden/{partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf => udf_xpath_long-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b rename sql/hive/src/test/resources/golden/{partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf => udf_xpath_long-3-9df8d27e31d96e0b35b9b40910d4bd98} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 => udf_xpath_long-4-3211913c56521887d30e3d1a50762b3f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 => udf_xpath_long-6-b695348ed3faec63be2c07d0d4afaaf3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 rename sql/hive/src/test/resources/golden/{metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc => udf_xpath_long-7-ed5af6d7451107a753b2c7ff130ac73b} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f => udf_xpath_short-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f => udf_xpath_short-3-c09b4ae6886fa58dcdd728bef45e7efa} (100%) rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 => udf_xpath_short-4-84f5f6bb2fdc0987d281d52a53a4b24e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e rename sql/hive/src/test/resources/golden/{exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 => udf_xpath_short-5-c09fd0565ed041c773fee9bd0436e861} (100%) rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 => udf_xpath_short-6-16ced3de15d4ec87a4e7001376551758} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 rename sql/hive/src/test/resources/golden/{newline-0-43392a20a8d249a279d50d96578e6a1b => udf_xpath_short-7-8ffdf20c15f3ed81bb5a92c61d200ae2} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d rename sql/hive/src/test/resources/golden/{partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf => udf_xpath_string-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 => udf_xpath_string-1-e315d11e9feb29177b5cb1e221c7cfa4} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc => udf_xpath_string-10-d87fb71039c9d2419d750a0721c5696f} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 => udf_xpath_string-2-110b583cde6cd23c486d8223c444cbe9} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f => udf_xpath_string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a => udf_xpath_string-4-302630fe7dac2cc61fe7d36ead0f41ab} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a => udf_xpath_string-5-19357ba9cb87d3a5717543d2afdc96e2} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 => udf_xpath_string-6-4837e470f745487fae4f498b3a2946bc} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 => udf_xpath_string-7-8e42951d002e3c4034b4a51928442706} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f => udf_xpath_string-8-fdfa4e17d70608dcc634c9e1e8a8f288} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 => udf_xpath_string-9-84109613320bd05abccd1058044d62c3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a delete mode 100644 sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 delete mode 100644 sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 delete mode 100644 sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 delete mode 100644 sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 delete mode 100644 sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c delete mode 100644 sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 delete mode 100644 sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e delete mode 100644 sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 delete mode 100644 sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b delete mode 100644 sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 delete mode 100644 sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 delete mode 100644 sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 delete mode 100644 sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e delete mode 100644 sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 delete mode 100644 sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b delete mode 100644 sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 delete mode 100644 sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb delete mode 100644 sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db delete mode 100644 sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c => union34-4-70479e10c016e5ac448394dbadb32794} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 => union_date-4-d85fe746334b430941c5db3665e744d4} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 => union_date-5-82eebfded24cef08e0a881d1bcca02b1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d delete mode 100644 sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 => union_remove_11-9-94da21f150ed2c56046b80e46da8884d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b => union_remove_3-8-94da21f150ed2c56046b80e46da8884d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 => union_remove_6-9-94da21f150ed2c56046b80e46da8884d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d delete mode 100644 sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 delete mode 100644 sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 delete mode 100644 sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 delete mode 100644 sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e delete mode 100644 sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 delete mode 100644 sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae delete mode 100644 sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 delete mode 100644 sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 delete mode 100644 sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 => varchar_join1-6-ab4392aa5ff499ec43229425ff23e22f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 => varchar_join1-7-a01639290aaceb4b85aa6e44319f6386} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 => varchar_join1-8-c05d7b534b51cecdc2ba2de4ce57ba37} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 delete mode 100644 sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b => varchar_union1-6-f338f341c5f86d0a44cabfb4f7bddc3b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc => varchar_union1-7-ea0d1fbae997b50dc34f7610480bbe29} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e => varchar_union1-8-f3be9a2498927d692356c2cf871d25bf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a delete mode 100644 sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 delete mode 100644 sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 delete mode 100644 sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 delete mode 100644 sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 delete mode 100644 sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 delete mode 100644 sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee delete mode 100644 sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b delete mode 100644 sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 delete mode 100644 sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a delete mode 100644 sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 delete mode 100644 sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b delete mode 100644 sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 delete mode 100644 sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 delete mode 100644 sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 delete mode 100644 sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf delete mode 100644 sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb delete mode 100644 sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de delete mode 100644 sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a delete mode 100644 sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb delete mode 100644 sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 delete mode 100644 sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 delete mode 100644 sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q create 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/clustern1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q create 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/date_literal1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q create 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_partition_filter_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q create 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_columns.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q create 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/notable_alias3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q create 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/script_broken_pipe1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q create 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/udf_case_type_wrong.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q create 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_when_type_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_vectorization_ppd.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order_within_subquery.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_create.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q delete mode 100644 sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala create mode 100644 sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala delete mode 100644 sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala create mode 100644 sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala diff --git a/.rat-excludes b/.rat-excludes index b14ad53720f32..ae9745673c87d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -48,6 +48,7 @@ sbt-launch-lib.bash plugins.sbt work .*\.q +.*\.qv golden test.out/* .*iml diff --git a/dev/run-tests b/dev/run-tests index 7d06c86eb4b41..f55497ae2bfbd 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -167,7 +167,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. # This must be a single argument, as it is. if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" fi if [ -n "$_SQL_TESTS_ONLY" ]; then diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 705937e3016e2..ea04473854007 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -253,7 +253,11 @@ object Hive { |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, - cleanupCommands in console := "sparkContext.stop()" + cleanupCommands in console := "sparkContext.stop()", + // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce + // in order to generate golden files. This is only required for developers who are adding new + // new query tests. + fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") } ) } 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 7c480de107e7f..2b69c02b28285 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 @@ -52,6 +52,8 @@ object HiveTypeCoercion { */ trait HiveTypeCoercion { + import HiveTypeCoercion._ + val typeCoercionRules = PropagateTypes :: ConvertNaNs :: @@ -340,6 +342,13 @@ trait HiveTypeCoercion { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e + case a @ CreateArray(children) if !a.resolved => + val commonType = a.childTypes.reduce( + (a,b) => + findTightestCommonType(a,b).getOrElse(StringType)) + CreateArray( + children.map(c => if (c.dataType == commonType) c else Cast(c, commonType))) + // Promote SUM, SUM DISTINCT and AVERAGE to largest types to prevent overflows. case s @ Sum(e @ DecimalType()) => s // Decimal is already the biggest. case Sum(e @ IntegralType()) if e.dataType != LongType => Sum(Cast(e, LongType)) @@ -356,6 +365,10 @@ trait HiveTypeCoercion { Average(Cast(e, LongType)) case Average(e @ FractionalType()) if e.dataType != DoubleType => Average(Cast(e, DoubleType)) + + // Hive lets you do aggregation of timestamps... for some reason + case Sum(e @ TimestampType()) => Sum(Cast(e, DoubleType)) + case Average(e @ TimestampType()) => Average(Cast(e, DoubleType)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index dafd745ec96c6..19421e5667138 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -101,3 +101,28 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio override def toString = s"$child.$fieldName" } + +/** + * Returns an Array containing the evaluation of all children expressions. + */ +case class CreateArray(children: Seq[Expression]) extends Expression { + override type EvaluatedType = Any + + lazy val childTypes = children.map(_.dataType).distinct + + override lazy val resolved = + childrenResolved && childTypes.size <= 1 + + override def dataType: DataType = { + assert(resolved, s"Invalid dataType of mixed ArrayType ${childTypes.mkString(",")}") + ArrayType(childTypes.headOption.getOrElse(NullType)) + } + + override def nullable: Boolean = false + + override def eval(input: Row): Any = { + children.map(_.eval(input)) + } + + override def toString = s"Array(${children.mkString(",")})" +} 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 590dbf3cb893d..c4f4ef01d78df 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 @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.types.DataType -import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.execution.{SparkStrategies, _} import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation 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 463888551a359..15cd62d3bf869 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 @@ -23,6 +23,7 @@ import java.util.{Locale, TimeZone} import org.scalatest.BeforeAndAfter import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.hive.test.TestHive /** @@ -102,6 +103,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf5", "udf_java_method", "create_merge_compressed", + "database_location", + "database_properties", // DFS commands "symlink_text_input_format", @@ -135,6 +138,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "stats20", "alter_merge_stats", "columnstats.*", + "annotate_stats.*", + "database_drop", + "index_serde", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. @@ -211,8 +217,20 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "describe_comment_indent", // Limit clause without a ordering, which causes failure. - "orc_predicate_pushdown" - ) + "orc_predicate_pushdown", + + // Requires precision decimal support: + "decimal_1", + "udf_pmod", + "udf_when", + "udf_case", + "udf_to_double", + "udf_to_float", + + // Needs constant object inspectors + "udf_round", + "udf7" + ) ++ HiveShim.compatibilityBlackList /** * The set of tests that are believed to be working in catalyst. Tests not on whiteList or @@ -220,23 +238,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { */ override def whiteList = Seq( "add_part_exist", - "dynamic_partition_skip_default", - "infer_bucket_sort_dyn_part", - "load_dyn_part1", - "load_dyn_part2", - "load_dyn_part3", - "load_dyn_part4", - "load_dyn_part5", - "load_dyn_part6", - "load_dyn_part7", - "load_dyn_part8", - "load_dyn_part9", - "load_dyn_part10", - "load_dyn_part11", - "load_dyn_part12", - "load_dyn_part13", - "load_dyn_part14", - "load_dyn_part14_win", "add_part_multiple", "add_partition_no_whitelist", "add_partition_with_whitelist", @@ -256,6 +257,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "alter_varchar2", "alter_view_as_select", "ambiguous_col", + "annotate_stats_join", + "annotate_stats_limit", + "annotate_stats_part", + "annotate_stats_table", + "annotate_stats_union", "auto_join0", "auto_join1", "auto_join10", @@ -299,6 +305,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "auto_sortmerge_join_13", "auto_sortmerge_join_14", "auto_sortmerge_join_15", + "auto_sortmerge_join_16", "auto_sortmerge_join_2", "auto_sortmerge_join_3", "auto_sortmerge_join_4", @@ -340,7 +347,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "create_skewed_table1", "create_struct_table", "cross_join", + "cross_product_check_1", + "cross_product_check_2", "ct_case_insensitive", + "database_drop", "database_location", "database_properties", "date_2", @@ -360,8 +370,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "diff_part_input_formats", "disable_file_format_check", "disallow_incompatible_type_change_off", + "distinct_stats", + "drop_database_removes_partition_dirs", "drop_function", "drop_index", + "drop_index_removes_partition_dirs", "drop_multi_partitions", "drop_partitions_filter", "drop_partitions_filter2", @@ -369,23 +382,30 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "drop_partitions_ignore_protection", "drop_table", "drop_table2", + "drop_table_removes_partition_dirs", "drop_view", + "dynamic_partition_skip_default", "escape_clusterby1", "escape_distributeby1", "escape_orderby1", "escape_sortby1", + "explain_rearrange", "fetch_aggregation", + "fileformat_mix", "fileformat_sequencefile", "fileformat_text", "filter_join_breaktask", "filter_join_breaktask2", "groupby1", "groupby11", + "groupby12", + "groupby1_limit", "groupby1_map", "groupby1_map_nomap", "groupby1_map_skew", "groupby1_noskew", "groupby2", + "groupby2_limit", "groupby2_map", "groupby2_map_skew", "groupby2_noskew", @@ -406,6 +426,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby7_map_multi_single_reducer", "groupby7_map_skew", "groupby7_noskew", + "groupby7_noskew_multi_single_reducer", "groupby8", "groupby8_map", "groupby8_map_skew", @@ -432,6 +453,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_sort_test_1", "having", "implicit_cast1", + "index_serde", + "infer_bucket_sort_dyn_part", "innerjoin", "inoutdriver", "input", @@ -502,7 +525,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join17", "join18", "join19", - "join_1to1", "join2", "join20", "join21", @@ -534,6 +556,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join7", "join8", "join9", + "join_1to1", "join_array", "join_casesensitive", "join_empty", @@ -557,7 +580,21 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "literal_double", "literal_ints", "literal_string", + "load_dyn_part1", + "load_dyn_part10", + "load_dyn_part11", + "load_dyn_part12", + "load_dyn_part13", + "load_dyn_part14", + "load_dyn_part14_win", + "load_dyn_part2", + "load_dyn_part3", + "load_dyn_part4", + "load_dyn_part5", + "load_dyn_part6", "load_dyn_part7", + "load_dyn_part8", + "load_dyn_part9", "load_file_with_space_in_the_name", "loadpart1", "louter_join_ppr", @@ -578,13 +615,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "merge1", "merge2", "mergejoins", - "multigroupby_singlemr", + "multiMapJoin1", + "multiMapJoin2", "multi_insert_gby", "multi_insert_gby3", "multi_insert_lateral_view", "multi_join_union", - "multiMapJoin1", - "multiMapJoin2", + "multigroupby_singlemr", "noalias_subq1", "nomore_ambiguous_table_col", "nonblock_op_deduplicate", @@ -607,10 +644,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "outer_join_ppr", "parallel", "parenthesis_star_by", - "partcols1", "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", + "partcols1", "partition_date", "partition_schema1", "partition_serde_format", @@ -641,7 +678,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ppd_outer_join5", "ppd_random", "ppd_repeated_alias", - "ppd_transform", "ppd_udf_col", "ppd_union", "ppr_allchildsarenull", @@ -674,15 +710,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "serde_regex", "serde_reported_schema", "set_variable_sub", - "show_create_table_partitioned", - "show_create_table_delimited", + "show_columns", "show_create_table_alter", - "show_create_table_view", - "show_create_table_serde", "show_create_table_db_table", + "show_create_table_delimited", "show_create_table_does_not_exist", "show_create_table_index", - "show_columns", + "show_create_table_partitioned", + "show_create_table_serde", + "show_create_table_view", "show_describe_func_quotes", "show_functions", "show_partitions", @@ -738,12 +774,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udaf_covar_pop", "udaf_covar_samp", "udaf_histogram_numeric", - "udf_10_trims", "udf2", "udf6", "udf7", "udf8", "udf9", + "udf_10_trims", + "udf_E", + "udf_PI", "udf_abs", "udf_acos", "udf_add", @@ -774,14 +812,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_cos", "udf_count", "udf_date_add", - "udf_datediff", "udf_date_sub", + "udf_datediff", "udf_day", "udf_dayofmonth", "udf_degrees", "udf_div", "udf_double", - "udf_E", "udf_elt", "udf_equal", "udf_exp", @@ -826,7 +863,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_nvl", "udf_or", "udf_parse_url", - "udf_PI", "udf_pmod", "udf_positive", "udf_pow", 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 34ed57b001637..fad4091d48a89 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 @@ -46,7 +46,6 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand -import org.apache.spark.sql.hive.HiveShim /** * DEPRECATED: Use HiveContext instead. @@ -230,7 +229,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * 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 = new HiveConf(classOf[SessionState]) + @transient lazy val hiveconf = new HiveConf(classOf[SessionState]) @transient protected[hive] lazy val sessionState = { val ss = new SessionState(hiveconf) setConf(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index deaa1a2a154f2..fad7373a2fa39 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.HiveShim /* Implicit conversions */ import scala.collection.JavaConversions._ 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 904bb48691e35..04c48c385966e 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 @@ -33,7 +33,6 @@ 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.catalyst.types._ -import org.apache.spark.sql.hive.HiveShim import org.apache.spark.util.Utils /* Implicit conversions */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index ffcb6b505b9c6..54c619722ee12 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -251,6 +251,8 @@ private[hive] object HiveQl { s""" |Unsupported language features in query: $sql |${dumpTree(getAst(sql))} + |$e + |${e.getStackTrace.head} """.stripMargin) } } @@ -329,6 +331,7 @@ private[hive] object HiveQl { case Token("TOK_SMALLINT", Nil) => ShortType case Token("TOK_BOOLEAN", Nil) => BooleanType case Token("TOK_STRING", Nil) => StringType + case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType case Token("TOK_FLOAT", Nil) => FloatType case Token("TOK_DOUBLE", Nil) => DoubleType case Token("TOK_DATE", Nil) => DateType @@ -854,9 +857,11 @@ private[hive] object HiveQl { HiveParser.Number, HiveParser.TinyintLiteral, HiveParser.SmallintLiteral, - HiveParser.BigintLiteral) + HiveParser.BigintLiteral, + HiveParser.DecimalLiteral) /* Case insensitive matches */ + val ARRAY = "(?i)ARRAY".r val COUNT = "(?i)COUNT".r val AVG = "(?i)AVG".r val SUM = "(?i)SUM".r @@ -917,7 +922,9 @@ private[hive] object HiveQl { /* Casts */ case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_VARCHAR", Nil) :: arg :: Nil) => + case Token("TOK_FUNCTION", Token("TOK_VARCHAR", _) :: arg :: Nil) => + Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_CHAR", _) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) @@ -1009,6 +1016,8 @@ private[hive] object HiveQl { GetItem(nodeToExpr(child), nodeToExpr(ordinal)) /* Other functions */ + case Token("TOK_FUNCTION", Token(ARRAY(), Nil) :: children) => + CreateArray(children.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) => Substring(nodeToExpr(string), nodeToExpr(pos), Literal(Integer.MAX_VALUE, IntegerType)) @@ -1042,10 +1051,10 @@ private[hive] object HiveQl { } else if (ast.getText.endsWith("Y")) { // Literal tinyint. v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) - } else if (ast.getText.endsWith("BD")) { + } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) { // Literal decimal - val strVal = ast.getText.substring(0, ast.getText.length() - 2) - BigDecimal(strVal) + val strVal = ast.getText.stripSuffix("D").stripSuffix("B") + v = Literal(BigDecimal(strVal)) } else { v = Literal(ast.getText.toDouble, DoubleType) v = Literal(ast.getText.toLong, LongType) @@ -1056,7 +1065,7 @@ private[hive] object HiveQl { } if (v == null) { - sys.error(s"Failed to parse number ${ast.getText}") + sys.error(s"Failed to parse number '${ast.getText}'.") } else { v } 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 e45eb57b3debf..9ff7ab5a124c1 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 @@ -34,7 +34,6 @@ import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.hive.HiveShim /** * A trait for subclasses that handle table scans. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index c6ff4ea6de594..bb79ad5538046 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -20,6 +20,9 @@ package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState + import scala.collection.mutable import scala.language.implicitConversions @@ -119,7 +122,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (cmd.toUpperCase contains "LOAD DATA") { val testDataLocation = hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) - cmd.replaceAll("\\.\\.", testDataLocation) + cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") } else { cmd } @@ -417,6 +420,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { 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) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. runSqlHive("RESET") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala index a201d2349a2ef..1817c7832490e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala @@ -19,15 +19,15 @@ package org.apache.spark.sql.hive.api.java import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD} -import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.{HiveContext, HiveQl} /** * The entry point for executing Spark SQL queries from a Java program. */ -class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(sparkContext) { +class JavaHiveContext(sqlContext: SQLContext) extends JavaSQLContext(sqlContext) { - override val sqlContext = new HiveContext(sparkContext) + def this(sparkContext: JavaSparkContext) = this(new HiveContext(sparkContext)) override def sql(sqlText: String): JavaSchemaRDD = { // TODO: Create a framework for registering parsers instead of just hardcoding if statements. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index fbd375639692f..5d98834c6fb33 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -45,7 +45,7 @@ case class DescribeHiveTableCommand( lazy val hiveString: Seq[String] = sideEffectResult.map { case Row(name: String, dataType: String, comment) => Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse(HiveShim.getEmptyCommentsFieldValue)) + Option(comment.asInstanceOf[String]).getOrElse("")) .map(s => String.format(s"%-20s", s)) .mkString("\t") } diff --git a/sql/hive/src/test/resources/data/conf/hive-site.xml b/sql/hive/src/test/resources/data/conf/hive-site.xml index 4e6ff16135833..7931d6a7128fe 100644 --- a/sql/hive/src/test/resources/data/conf/hive-site.xml +++ b/sql/hive/src/test/resources/data/conf/hive-site.xml @@ -19,6 +19,12 @@ + + hive.in.test + true + Internal marker for test. Used for masking env-dependent values + + @@ -28,7 +34,7 @@ hadoop.tmp.dir - ${build.dir.hive}/test/hadoop-${user.name} + ${test.tmp.dir}/hadoop-tmp A base for other temporary directories. @@ -42,22 +48,27 @@ hive.exec.scratchdir - ${build.dir}/scratchdir + ${test.tmp.dir}/scratchdir Scratch space for Hive jobs hive.exec.local.scratchdir - ${build.dir}/localscratchdir/ + ${test.tmp.dir}/localscratchdir/ Local scratch space for Hive jobs javax.jdo.option.ConnectionURL - - jdbc:derby:;databaseName=../build/test/junit_metastore_db;create=true + jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true + + hive.stats.dbconnectionstring + jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true + + + javax.jdo.option.ConnectionDriverName org.apache.derby.jdbc.EmbeddedDriver @@ -82,7 +93,7 @@ hive.metastore.metadb.dir - file://${build.dir}/test/data/metadb/ + file://${test.tmp.dir}/metadb/ Required by metastore server or if the uris argument below is not supplied @@ -90,32 +101,19 @@ test.log.dir - ${build.dir}/test/logs - - - - - test.src.dir - file://${build.dir}/src/test + ${test.tmp.dir}/log/ test.data.files - ${user.dir}/../data/files - - - - - test.query.file1 - file://${user.dir}/../ql/src/test/org/apache/hadoop/hive/ql/input2.q - + ${hive.root}/data/files hive.jar.path - ${build.dir.hive}/ql/hive-exec-${version}.jar + ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar @@ -127,7 +125,7 @@ hive.querylog.location - ${build.dir}/tmp + ${test.tmp.dir}/tmp Location of the structured hive logs @@ -143,18 +141,25 @@ Post Execute Hook for Tests - - hive.task.progress - false - Track progress of a task - - hive.support.concurrency true Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. + + hive.unlock.numretries + 2 + The number of times you want to retry to do one unlock + + + + hive.lock.sleep.between.retries + 2 + The sleep time (in seconds) between various retries + + + fs.pfile.impl org.apache.hadoop.fs.ProxyLocalFileSystem @@ -194,4 +199,21 @@ The default SerDe hive will use for the rcfile format + + hive.stats.dbclass + jdbc:derby + The storage for temporary stats generated by tasks. Currently, jdbc, hbase and counter types are supported + + + + hive.stats.key.prefix.reserve.length + 0 + + + + hive.conf.restricted.list + dummy.config.value + Using dummy config value above because you cannot override config with empty value + + diff --git a/sql/hive/src/test/resources/data/files/ProxyAuth.res b/sql/hive/src/test/resources/data/files/ProxyAuth.res new file mode 100644 index 0000000000000..96eca8f61de33 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/ProxyAuth.res @@ -0,0 +1,15 @@ ++-----+-------+ +| id | name | ++-----+-------+ +| 1 | aaa | +| 2 | bbb | +| 3 | ccc | +| 4 | ddd | +| 5 | eee | ++-----+-------+ ++-------+-----+ +| name | id | ++-------+-----+ +| aaa | 1 | +| bbb | 2 | ++-------+-----+ diff --git a/sql/hive/src/test/resources/data/files/alltypes.txt b/sql/hive/src/test/resources/data/files/alltypes.txt new file mode 100644 index 0000000000000..358cf400ec815 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/alltypes.txt @@ -0,0 +1,2 @@ +true|10|100|1000|10000|4.0|20.0|2.2222|1969-12-31 15:59:58.174|1970-01-01 00:00:00|hello|hello|k1:v1,k2:v2|100,200|{10, "foo"} +true|20|200|2000|20000|8.0|40.0|4.2222|1970-12-31 15:59:58.174|1971-01-01 00:00:00|||k3:v3,k4:v4|200,300|{20, "bar"} diff --git a/sql/hive/src/test/resources/data/files/alltypes2.txt b/sql/hive/src/test/resources/data/files/alltypes2.txt new file mode 100644 index 0000000000000..c6a05a1dad893 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/alltypes2.txt @@ -0,0 +1,2 @@ +true|10|100|1000|10000|4.0|20.0|4.2222|1969-12-31 15:59:58.174|1970-01-01|string|hello|hello|k1:v1,k2:v2|100,200|{10, "foo"} +false|20|200|2000|20000|8.0|40.0|2.2222|1970-12-31 15:59:58.174|1971-01-01|abcd|world|world|k3:v3,k4:v4|200,300|{20, "bar"} diff --git a/sql/hive/src/test/resources/data/files/alltypesorc b/sql/hive/src/test/resources/data/files/alltypesorc new file mode 100644 index 0000000000000000000000000000000000000000..95c68cabd15e0ce34d472d552f4bb60193614b6d GIT binary patch literal 377237 zcmY)URahN7*ENjd?(Qtyb>Z%?(1p9ZySo>6clSam?k>gM9SX&vSkZky?|-ttljKUq z$jr>a%#=h$O%fdf0tpri8sQrqI|Rg0>b{wE2n2;uV!N9J3CSZAH`YymGb9Ax|Dc2a zgTxb!-6VuajK6?LLjUIIUEhlFAmsjo)qejUjF?#DCNV^Uz2^8a&f0W+oAh;w73Zhz ztZR+_|I(5U|D&CL^}o)svD-dIi@sw8uR4CcaN|SB|0iwn(99eHngaGeJ=7NSF^mQv z#Q{0V_k8Q(se=^0=yUaJ@DOVM|2*UW@c>@Kdw=0!a+o?q31?N@(jD46czl1_r%*vS z{Kw@y{XY#!|2u&Exj_#coQQ8Et}g{4|7;FO!Yl(f2u>zShOjT{_jf!aJdlwt9_w$= z|EKfG|1D%lOgfY=HK>P!yAJA<5X!(EvnolXz>^u9?SvruPd?W1p;-tth1GvL|G#5n zHwfDQ;U|YCAy5>`|Lw2GY@j%E`2Wr(9sX~&@dZ)?0^+T|=<)08P3)cQTR-%jGvE_> zW9spvA3NcpwT9ivyvwB<%;400{x!4vX5qW|ai+X}>0c@phi& zJM2xuV4eqQP;wQ!FP~>`hbm&~=~7~5#c9bH3z>f88u3nSzu3|H2N^Ge0u1~ZcJ*m; zdU@a-A4&=@J_s$I4I38|Ik%2s(yBFsA10W{OEV_fedq1cE84>T!L3Z7%Xd&0walH* z$lY1Q>^L+%&}A5k7#RsymR#j$`sMt&p(j|izTFG9t{ECvk!J^~%o@(i!K*NW6we5d{mG4WvGV43blM)x;pw2Cupolqfr)}o*ZQ-VA zm&lG6+Wmv6-qT~69$l|;;g@ApeW$&Hj9HEU`!xMo{f zTNN`6q!ZGBGHd%ARKK*(89VTE(3}H_1HSPr!2nDd{ErMfoy9Ctc&-daBn|6*Riz(o zw&2q|5N$U@P=@^TN2r@1vbl%tq~w$IF9|`-ib%}QvMx>g3vT_?fBW4wM4!bmI81yN zfpW$cIVu&aTvSiDHUI39Ww6{lv+OJrcy40+Q`y;>zj^#lFKpx zXs+O2NA}bJ1h5m1?h-#Sca65Yj(LV5*y*F+FC^U(>{z^i|Hcb=1939)h6OfmiotCB z&8dzytl;@<$Y4YM-B_#12Gnpth2nP2FEiEWgxJz?WM36z(vL?`q`Nzvlp;1jvQv-; zuIC#JG5tW!_ol-g^=hOLGo_z8k>~@6-;5aQ(Bq>Rl3;JSkuNWwl1dR&LsH0$Wg>~Z^qYv5!8{=tkuE;ySes`B1>d$0{vw=f-Qj8Z?H z9&ahWDj)OO-8|DZs%>iy=TAYq;L7|bzhTql6DImJ^z_ZcWAF5P-W}B{>TOgV5gq)! z5*;T!8+QTeQcA7ru>+<0YOAw(DqqK6h889kEx5GOGOf6zV!8vo*eRheFoRQ_c`vd> zbiKonFMJjVsZJRJN_G6A#=R>KSCdEmR=`N0b-yrU$34kBltJ7{f6z0o>8u^U&Kgve z@zX&Ozp3n=iOK-Uwm;ae9(2t&@t{YSzR>eMy|37g$e>&`YiS+Ts~Yf@AN)21Ss=** z%yTgwNP`+g0R(2w24`FwxkgV=gewE}e6&QMhY0($k3wqQ-c8^PYv^Wq{6|RcNAmt~ zO(QE^D|`o6RwYtS(Qf{hEGOJS;y@tC{CP>D5v;2W9krjy4cBX3~oKWb-jTl zzSNCE4d5+0tG#i(5MecJqB7Q>g-W}mm*v^6^qNxce3l&0byu*2>Ea|tsi&*ZmZ=U3 zht2f+JXETD(FD4Ldnvj59;NDk?-yDa$_)sjrG{ zkj%X+1?M4jSjP;eG8X^R)*LJMwmkt-s@7qQ*-<%#GteK;V+P{||0U5eDLBX2f$ooE z<|2--K#h>kOpCnwTfS6aS{0>Ml@~|aUro^=RI>X09$Dz%XHg5jewrZ%f=%5K4;>8$ zm&HBB66^+YHni8ue8rDU54_T7PLd-}ZQ*I-mG)mO6}wk7^PrY*btt21)>neh8N0$f zpJ;e#X*wuZ#eRz@HaB!>BewHKPQ+UM!jzTa$7%l*^?Zx6>_I_?nqpiDVLV<)CyOVb`06*1h8@~BsD zuvErN4Q>P!Ka-Y?N>y}nhZB^QH8PZ=kF$&A(9OL(&8sz{#8$K7PvY4OP~qK-_?@}4lLMW9(-g}4)&EB!^hD~*`3bVNP4R3*EvVO zT4^WfU1w9Y4kR^&227$qdDv0*No%P8V#nR2PLHXwQiDPT3FJ}i8}2I0pL#2O6>-6P z>b{nM3_ol+XVu(uJkfPso)#fBg_UfReyf00?+E<1XR_wgYZAU)&Ul>HUr9<<*Ah9g zWk6gFH>j$hypQQ^bOht#I=+a^M$}p-8{O9kw^mTyMK}!!m2osjNXTHkYu|l+sH6D1 zR`$Mwx|Z@wo~QRniy%`<6#B=)>@gT!-A>fWW!sITUv8ve$|Azm*?o=5yd?6~Y_6P} z7(M7t&ig3{V!jwzNK8|3s9BsYH16?*jKzIOc9mN- zw0ETMceQlHtJ+UNf=T2OedwPGs_hBPBIi5||KbdYlqe?Nbz7+-S;6HcM}(}J*?+td zB5@h|vfiNm7=SY11_hZa*Z~@(z2)8AZPtTkA|Jos zW9{VfO8BJ>;0Zj5DWO3ymx#T|DcN+tV~JnE++jtO^iPk z9q#(!!zWtz=Y0$Q{276f@y~9;2Cuq&EOM37)g>oXa?tsC-qoWD0zf-Lbi7lVqFp== zmg3Ll^a0j`sdSh3*LERS%%*If!vTPL_`8AEtB8zoU)eJyei;W=eBJ{9-MUMWw9xh^ zibs``r=!fge@id1_!ldGj9wk&HzB=`%Fa#T3tetjn3AuZ8Q||Dy=9z$85sfQpNpVn z-=mVb-$;$uB(uKdUNC~&t*Yj0+)r;#j`kFlHbf@FKiK)~%in{uN6zviG&gH76orN>Hcn zQ)<5#z^0*URJsDGPcZ~K@quS^6+7F@$CD<1~{40qoCggtnr8P1#!&b)-pUsfY>!wetsZ^j#nz2z_KP^o1AW3-&O$$v$%(BjHFHN+jwOVUBLsc@H~U`o+}C7q%oSzvOOdeBLR@8p znTcv7RBAaw1?l2J~`V@0dY;C=*xSk4!4?1AQN2d|)kgscJLQP8 zU=v2ZF44zWsmvQ`n2L8N!VsjcbD;ZN=hjyiGg9_1{gO}X%CNcJ8|J3>RT=mbO?;p% zpLa5yz6q#89Dw-&hTeF5rz|&xfh%DYoCYqC^A&dWe~iKrNm(CNYT)kbwqGYM@kqwh zvSZK3Ikc_d>i!_-O9se;g^qc8vi4M|5FJ|DMjz|UAv!9gotf*I+UfWZ$TIJ$JW2hv(imb59GU zmcMl@#Hldf)G7#iIgt0 z=VnM~XzuaF6-#wf8uS(=BKogxM?aDd&THVuRQ1;-IJ~>eTFs{}W%#K+M5><5j2pDE ze(%Fg<&2u<`c*i(8FOk%m?{>LE5DNNh>sqm^acItPIDslvGw8#yE-naHNoeW~uYA-yYw@=ruf1 zs|Uo1GGA@NRKMlPIBxui9>8FBUadC#YFy_AyK7a%-U@zq$b%P0Nr3HAD?9_va^MeE zZ#;4WOr@O{a}MkJ+c5`BP)8UBYS__tYCkc_7zb+Om#q{SZ!%4p53hTRH62HuK8pX;t}zq`X*!Ty zV`*o+KQ<#rNP>S2dzP$NkCe(`M3J=`{kc(z!f$Y4GiX!wNrp8hj!1NyiXn?*muuECD2nq7r4Zb}D{MN`};2)+Z#7V`3K&%Y+eU)6&_v1)w`K| zude(4bz78~_X|C<$c`l=u{mFK$w%7@?I3hf*tmNAut@y__NV!1?$T$$BizFAJmuk;#w)uM;T3K#OLFQ$nV(H$L>6a-$HATOGHn!CBY>>HnSXhVG+-9 z>YqR11(~;qE|2$zJ5OKLm(F`9Hw42&%&av&lW(|X3`PM!x*sXlB8%^F*^KzTtIEyi zv;;}|TaMch=U|~8NE7WJS9%c|%favE7J#!D`g$W8EcTX&b`R|G(c(*0e2ch}Easkb z>&t7hkYuy@8VAWwbjTybh&Pw=Xz{VMMj(2UEf>S5?VT2T|CpKvSwxa7?P2?l1u0TU zMrb6)d6;xVU*gfsBIbLZ#$$2yIb&#s7rZvNoinfJ*S*)8ZiF6!-#lL_HI7_Mx2|FR zvq9KEyfrU|L>M)6%Z^zLi^T=Ol76&ZLMC50(Z*ipcW42g%T|k^C5&4sXVdXmBBR-~ zcXBy)#7_9KIZ4UEk5{uZx&Y+*nz?FaA3i&P$K%%Bo1R=MkNlvvz^^i{_4R6T??@@? zXt}W?CF07z*byZ^H#_Ke3=)~#hmNfhS)+dU!Uwcs&g+FtILZVd_FwhucnT$`i$R!# z7Sh3}(68`W1XN`5ToQ68Mv8V}9irY4X3`zHgox3f{p-^I z51Bn0D7T8Zo+GA?U-`W2!SRhV6d6}v6+9?2t%oDXjBm$__g4?$S^(`07ZDPo0AcVe z2^!-68pAJcW6PFkWj-n1F7Pk$6*4IkKPAK*b|NS1HFkm|3uW@{1BKE$;De72-5jqw z*p~LO96(3ReQOj;2DPh*^LF-)%|lNfa*LHdjYfIJrJJ+TZ8ZCJ{WWxF;vR-cL7Wr8 z?FN!#-8nhw`!j>h`7q3|LA?HMy2c7zK?keC*OzvH|91&ka!I$eq?t{OETe4|rt>-{ zgAOEbw@Fp!n00gmilR`LLhTj#jyyN{@YzA-7K*w2g?Ig#1tVE=dz)3* zMic4j^E5^{v(Z|mpy;Wxw(gY{{;+B_ny8`2vYmb<1|1+JHEi*f5t(IIm08ffkgi7M z@<{LTLe?A(dNY(@rm|xh`Gw+{OP{fP07a$p?L`44fDeQR+x%$*YgJLG*E2->0|DU$ zJrVt}JEsPRZrcniiwBl6q&2SNJcacSrI7%(A^eZJ*<4%tmB|u(klxKwV5KBIg^Q- z+N(_Eex^oCSX!{|6rToo7y`QI){-> zp%)s0Hw5zoQS!F8zjbaIe=|S}X6r~@C%>Y8qbs3!{Y7hKlO7+ABH`pKI#3PJI5jwVoOzk`d#$E77uy}38J4jVG^gdiM;TleC{r!ZTUR^1|i?-^z8xfV9;TOWX;NX5e(_r!hQL9EXwtk8X z6OFHzpq-vWpv3?V$X`J-aZ%y&KN>ss2nC39dE55QeSH0|f z25{A>sD6)&ifnS&P<39NbHw;OSr6oU7t?uHv1?1L8j z!ciHX&eN9stD-GdU3P-U15EErcN(4Rp9*?rSZ&|WQpfOP$O-+z;cb?5av)iG9x~n7 zu&SA}4r%x|y+qZp)K>W%Y>M_+^l*3_)9(@7+t-~p4~zw7`&_5s`wZMcyIwoXS8JJNvoI{?!NhA1M#0HKu1w zFi>~+VO~2G>-}ll*gV%KiV>(rP3keQyAsUe_yAsGVHCyhX&riQzUhHzoegbXZJAqu z4T9{#mVcfT;dsRc>#c5`k0u>twoL(|T-ROmX)DEEk!hdf`4hB`q1(Ut3-4d-y45h` zjf$@zD2i0)>Obr7?W{-$McVU5w3vF`(k{z>*VvLYw)E?Jj*bxjiJZKIX9y@Tuh8J(P#!pX#*A$xc zG~FUnIX-PSe|~O<%d=>jFi~eSb3*eLMh-NX@O$hqquMavQk9r*_lW^TTT@wWh~Cy+ z4^?TNwXLZG_((D^tC?MXT*rI^l3I`zQ2{tIjIcI8(tt`1 z2!G-g(w?{d1Gs&g#|MH|Dq!rz^VgPG{0Gm`MN9 zXt!6^4O>2TIO2}QJ1cdAke}4fI#SPGlD9(`G2<$b|GHgPL*NVT_N)SqOxua6*vmeya~fBVt9>RwN|->_n6 zgX=lxo`PFp=UW&$Dp=tM*gbe)dNquKU@#CS{T$4@tt{nT2ta3kGkrry+ks7!64k0b zWbgtkumzT<(*UhyCqCF1K{AROC}}KwhUSvwa?Ko~!+7K#SSnf_p6b?bm#dIUX2EeH zr3UVG$5#JY@II*wAQrBAu&S6@QIbW~Fi27+xQBh+!_|bl@F>)~d(Km9bjEZ`vG?QA zC|T?5mh=$6ZnbhPV0Ns>&~TyGNYDUO6+h%po=4{EDlr^=K!MOo5WP+D6*zEzv8D#4 zMK#BERPc~>;D-98rEnl}@~3jXXLRmf>el$bfB=-XR~4o8aD5uP{5c_%y&CTj9hjCT zyEB!j0#mHe?J_|dl8c;=VY!_{WsycA5)*|g)CgvO?knkI?l0L(UhMFxOXOTW&tBiF zV=I(=97u{*-bAdaIU_*=^_jP%-xU=OV6Er=-l;PueAC;fQMW4;g=6lJBHlo_IT@l( z`cY34{?{jPTAj`QaG$`@C4;sca#^H+5xYo4Mqw!IAiv;16)_KP_lyUvD^7fF%pg@! zMEDeU4RX?r+(i^*IS*KXQB`E9LF@=dNB5m%(*JSQEAymn`I5io zngdz*>!#}T3D^BP;TGHk7P^A@+eb|hgBKUm*JOh+maayc%9nOO9`u*z?wg*H)(5bX zdZmpw++fc&xxndA0inSLoR;V!xRxth4cc3uX;z782GKR;I|~J~gi^?)ZFT@3 z{CZx3k5lX*!}HjChpWaukI=qCvgT=`ioG^)ueoW@FjVc-uPj8^)u|UoO(BmGr%Sw< z$ei}`x5#hxaURX{()1i3715AJYS(IlRq$(LL)Z2=D7!lqf z87OgeKmMzXno7(m9!+FD`okxc&QL7w@>+W&i47KcK!K~vGp15LdZ!$TFBt0v@*Bm` z?JnuLFB)hkPY7jeDDdwU_#;{5fBjN2$-4?>c^dXEu)NS4)Pu~!F-_0pW$fnrE#1Eu z9Ri@OP^}7;NWvz&ZT#vHlQm(}FNPBeZR_-(CEDBzBw;k;bByQM-I^)=0qt(e;jQ!vrs72$@sTO~%8-LlzP+0b`+=(BD3CYJCqlpLVbTqg8O zjVI}swriM3jmae)tWX% z%DRRRV`lf(v}OgM@u#>J$t(^<<+zhl$cCJmaHUqu7fm01}~(Sc5AZnSuUV-QhyUQ3xqY{#tPdl zE26(?h#i37T+m>A;64qpHdVxs>8iYIAPmNx!eld zP=SYKH#Q(K#lvbuQYS@d5{1eStE%a=N3W>0C0KQBEjzSrqxmta1D{XeCBBTsj51A; z%j^NKeo(qbCCnEyUIJ0WPr3PsfzEd87CItX`l>Y7l_;hzmvInQSKe^z=34F|t;XlQ zT3+9JtI>y6>Ps-7&AfP*)lENi%hT2*#!7ti)7R$$h|vyhbBp>b>u<)X??eA*$7jpm7TlB0WH<>uN=cAg9Axlm zaEpGd{Wj&<+9bC4R{)l0k~?sAnx#3$EQn3$oFb0{QdJaz}}&>E$1u|2HK=Y zaXj$v6*+4=PlE(K-Y$PE0ie|(sG$d)84rR2x5Tn|eO90T)`+u%BG}$ck(QT7ZK$BT z6ps!j-KHP?h_n+*rMdw)^F7#uLlHXgegsYPHH%tG=m3 zY-#Y9R=3Ug&zfxs1!H(;YKzPQ3<`xul|2Xy;##Hro=LJNTYV6=8;;cflZr>U8QiHMN%sRLCAS%HBwOytcXZ`^ zx@3c3pR{O-;xwdnr4-6G&9yGiL*k-^1_Q+J3NR9>)DS8@E}He(v+4fw6Im5 z!(5`9QkV5ny+9%7HB=EtgV5_T%1uj}#`qM{(G6#WvfjMp<_n-#F6r79EiZQXM)ubS7oZm2bgF^2Y) z3j2#{J61sL3lxWd;ECn_oRo$VwI2fuTDr4>CBP(*v1W>mtP)@dIdAD@#c{H5fqVbo zdA0`{(o&L<3P?6%Q6qGRaQV2Y# zl?T9Q#x4lWI~vgkI3#6hFD+`?v^{297B^%Gj(qnE&3}4Tq%i7Bk{+m8D(qPH*^x9y ze}r*pF$%R_5^S)Sx8h`3+A`$pAHxA=8Tk4Hdoe;}b@l9-{TkY-T5(eM+ahosZDcf= z@4X5i6q5$}u<5>`JNfb^3gwW(()6Ekb{;y4Pid3 zj&#PKrNo-rhY}W-o2W#1n*A(AeuI3AS<-x7#4@Vw*|Z1^`@VRrBKh1xs_GaF{*2b9 zG|f_>l53Ji_ssaCe&x2d?O0mAkEbpk-vnJ5ZXgx7c*#7n2`^+p3rwuu>;0Ue?=jS9 z(rM=U(~>T>jD#vY|8{6`J|NtuJx4fx?7lJ_Ywvk*A%+D#R1XtB{VG#V!QUg?#3k2R z)F3tHr?C>+v+}lBga z2L|eUAK**x$5kOLzh7cXv}d7z46nl{)4g-@8Q+L!NNN*xTn4l!t=-#SP^wYT$@0=g1*IMry3LFxVi0T3Epdq%_h}N(6J;=@Q_+ zzB~%#l10k-+$UFkH}} zW3lt(N8X^fD%*CdpORq~b|FE$w%7JY{d+3LyV$C+DYsx-0IGx)bNXv=621v;l39f7 zlAHDao6G7SjgN%G3~Ck2);VXiPZNZbx)Q->E`Md<56H({ar74mUVVpc9kDf|yJ*$; z9u-1IaEzpF_t9s@SEr?~1st{wv{N980josm$j#!!)4b#Mff zjdJ}ad@{tw!Xsp{zHI1@4m2UoI@Bokqs$HLlAl;mHnPCC2wq+pm1^ql4`C1xun+dT zi{CJMBj>~zdTfF}DZYdD4i)^+qt{rsFQSneBrZ5rf;M17)CF111xc7Q(i;U5M%YAq zkBe2M&Uz%GYik3fBJ3mxYL2{XL1y$WFmKvuwbJUu&5Dv=X~q)#x@Q)#BY!B37C1HL zn^cTqFE#~q4rsOg9ohN9h$tk#Tis)zI9*UJ&yE8t$7QH2h2)W%_>e3}8t%P~ z5^Ow%r*%E7YXnpz*@EOcbnzus0kX;mbT)XZ_+$Rb)8&xurz(~)&RHrl$u8-xY)LO_M;dm=5TIaYzLd+;?!Gsrya?TE zCz<0F;+x&)X(g82-%7qb02bCOA}?hdr7U$*@jtD#P~RxXiU-O+Y|MancvY4I7!IrPX$BW6`9g}60&~!Ai8RB3m8x~QRP@`e^0>1|KNX@ zrNE2R38Ei;RJN05Ui3b(;zBtgJPYDQ=m3%&F!M}4U0C*1Ui#lTX+V?6y$It|`)m7@ z8FSxAC$coouZUu%89Hu-?*SEP&h0|5B-UT*b7iSzvTCX=SLi5p#K$Z8JYhmPF+| zv%*2eEJnj)W;f~IFErgwl?%Eobf%6cUQ}H+(#PdrHejcq9)siW+SYsJ1*>@MW1{OZ zli)_q8F1O@s8L)LDl-WVba*2DjRZZvD6%0?G|EA<*o`LGH<4#Oy3FT_mca~$yZXW5Dz-2@_3NKPVL#!+VKF3U~ zEcbmA`n4}FO-o%fyb~u4w#yF`j6aaXsI-TYM6PB@sM9!+AglpVL(E$u1!_gj53H2 z#ZF7RNdL#Q7^nM&i^I2xIxUGuLG;fg36s6V6j$ky8OLZoX|Y%53Zs5( zOywq4O#jMXiyY#8p}NYOu?k<}qX1^nEW-`j?@V2WIFfCKlA_2skRl&4)9euuAZ4_V zMJ)|=9zqkXzxujqn0$v}#lh7n#r!(5;kHX(We$S~lioG_|CAXgI#{#@TRnCgQ5`fx zp=_2*pWKv?|4-uh@^Kui$Kx9{a zBI4g1(PJvbyG(`vu%XIP%=eR8)LxiiY5L*Z%;&@fwH_@Y4*aiXFzr7KtwehTm%7t=p>=IhVxcrfBsqe=f)KR& z!LsBrBpKVjeA#f#%)Ciyigb1~H1!R5Flupiw)cnW-bMV-xSBd5lUs5#&X4Xr?ukkr{_16XBatw@>V9j5`drl zF4xMZAYMzY3)AU-rURyGg=7lAQq#p##mTA9qdc(=XV~qS2y*^OP63SebQN-%=O)Lr zR>C+ZR^I$BXkD4_NJat`u6q!b?nMmAhK5)q>-!y7n!qF3iyMU>k|n+m8vWgZm3f*s zicnD`mGbFJ-sZOaT6T?c9OgT_v(}SU32-_Ijb*&OM}O4HnuAka1wZ{x6u;+RnC1@U zl8+yUJA52bvD7a|D4g4z1KGLLQ1e)#pSq;qS^z#;!|EfE4-6C+$D%&IaICsJowU; zo~iXyiCo|Vt0b?jjgwVpjd<6GQ86e5BjrSH&~M}%>TY))hdJo*qC#3 z{{VtUvqTzhmrQRG_h=u!UTlw6;PF*c{snh^J#%NJ0=azZQ}KLE%uStl$Rucv(m02I z|CHK^{IY~)6TotVlpd)(?2=W@kcQ(WYrE6dctdRi&VXBwTe)VYIH{%0 zc|Jf=Dfa;5ft>Y|Iht#9jQY;X43eAnU&b-E3i5;?Qbz$9H1x_BMVxx0bwj>Lxc^16 zh|mxvY=ro_EH82qTe-Xtdxg7r9NvZdU=^R}BT-)h7~Rdl82W?TDRSCR7iMZ&S!9O} zb!R1JOrv>1ceuD(l)#t`1iMpbdd}*u2TtPQ7VT$dOgNaVzp8;Lw!ukLue|*hywZP9 z$^xi7H~ABMz+6Q3LJ&5C=_+R1w{C*o|3uS9is_;p8xXKCQeb%S^jQ2Kzst8oUi(%{ z1L$wORXAu3<7kU}956)X@Q`xNws0BZ5(Cq+^8?qtzbxuXhgIRBGrh@SW1y#;MwqSP zK48iCXbjH8HfM5%SRr4%h}Gf6Q>FX zMH$+^fJ!|%^^o7~9lAbcOf7Q;1_6G^f98a7ictaA;O7i_jBi_P5%7s4u>Gq#+0 zh~;$;f_xrY9$G)TjKL*V7|W6lgq)D5jE)Lxzl8N`4#rC_q2mQAh0-iM9`P~ard6{F z1^wTcQMr5)W5N4{AWMgs+aNjT0y)dHEA^EbKWD3`egwwM0R0ZhMr4gXP7FoC2;Dnm zYS2(mfDZ4Q0b(z?Dk@{+f$~q{X>4QJ8RKJQbhv>ZGa9-2#s;{&Zx9fq)`QwBLBll$<<3PmebVh5`sk!bAuT6|)UdI8#q~+{2^> z@%oC9b^CIXDF|j%#X+RdU~$R{i;*y=tLfawG4ixl#DZ-H`2-kB?|f;#I~&K*uV`=m z7XoBQx8be)TH?yKLPHSGs^(WwYBr3>kq1sX)!PUIQ#J7ub=b;OK}^*q%82Wf$6oGy zL7YRzL*!*w;*JS*`rZv=4TLk%jw%??BFaH30BotZJoQHxYg=M$X>?=I9mWMB3h7ir z6`B^Cn3$L=vR|||E|T3z5)7#8yPGWmw@GC9FA8|4?La!0?|{PLUC0{dTB!8eI2Dyy zatRx^C#zNSBZQUpF8!m&I#lhpEI6SKPePBp~+hr=hk5IXOeh zh=UnQIP)PZp(`6IJ)>Q`sWZhvh&G3VOhEd89R6jxn6@!)v4Wm8H2*9mJ{7Q+$0ZiL ztXqwv&tx&C2HILQRWEURP^9#_DK_YDhxbgX1nd!@US=$JsJChuMMD8Iv zs6W<~wVwRw#ZY=g6mD6j$+#5%N-0jO!PT%Bu*hR!8M7~Xh)t$(QQm4G`Ara| zG9(WziCd^Nd0d2Jo3Mo-0);Cr0WsN*Orz(2?f$vc8?d+QKzNlcIy**8YZ--&`MFri zP$95iK+wrxS&5P#1SMP$x?GG$lZezz8;v*%LNXR!LQRk*LJoXclfXx(BWLxZNLO}h z8lIfU0!%e2gS=4j2p_vEG@R7IzGeDUG8aGOX@t1Bl%#WKwOsI1xPZIFVu%mR4z^Q< zINgj{@J;XV0Q>tbS{f1o#*iEmDMP{ta$-UX|3_&?@E0M_2v|76Gx4eHm9nDG5Y_!g ztx`Rp<_s`^IV+e#U6|-Lu&!JoDnjA4Oxe_D5*{YSU3+ujC|Bz+;YN@$I;>=cg*1ln z+vIa`#%gKD9vE9i6a9dN8t*8=XO2wdp~Z=3uKaz;>bqRP747IwBYQ-QC5UrweOlJ7 z8T1}zhZE}$UpJ1eUh@`6MxuGn>5k+@cR*VQcZ;E8{l+8kKQD@|{Ki2!RsDjpF*6~l zgT2bM6=MVeRZJlD)%8gH;pNJ6gV3TGlJq^eu>3=_mbHzEGmKvnM;TH+^y!81;hxgr z&x2Wxc3h(qzS!CH+r1H%tL+Fn)c9Rl zqQqHklt_KBUSVTPnvXTz00V>*l{OSlRgAym6D7y%p+*>fA{9T)6rvw*CSYx3f9~u%_D}vY|8(akSj4ySKK((90M@3H)slWLL{PVm`_c?oT z-bUgT@};Tq+Wl#K+l3$`V4#!qWhg_?=Cvry#FH7kv2xy7uo!gN;l1I+%E@uK9|T3= zzIzk+N=sDYc9m_4-&kz%K&af1MnELE)>sXp8SHxh&y$LM>k{)p3D?l>2+_8FFa#@- zs@yrfiivy4%(a7L0;#kn#N;G8rp5wQ;gL}G3oWEr23le0VG;3}*vnOLQES~FQg^Uk zF?SSkou)*7)DntI;N%K#lRx|6ZQ8V{9|APIhPI;*;$g+&*EnNcl?7bnpBv7n=*@$?K>Jllf4zDqCAKAzCk-ykoOb> zf#oigP8zleQ+kKSNV6tK=Aygcc>6yzTPhm*lGg?DYQt*NJnDWG8`K1V%y|VEc6eRr zg~r}j@WEGUYa;}1m&Yzx#9?0@zemV1|!BGg8Sygt5OPF z0o&6PUfZZWSn5zmb^ChzW5p=bpq656LlP=NMN2c$FAGW=@+R>Zz>NXi;CvvIEn^*=CSH}jT!p2 zXZ#{aQ&%KP<;o}E`;(`CI&Q8MVwsHVZY?7f$b=0<+b__3iFVbLoel-gxFyk<=R45aslJDW=gd~P;ciphh6SNc9pW-HtCoo4H1RnGI{P5~r~|=x4&8zNyIe5X zQ__sO9%$vs)wpF3*F)fU%tv+K(MWC91RaT=-2k%R;8a<%Yp!Jwg}}0)e;Y(efEp94 z@<8#K+yl5|>CC`qmja8!{T=Ft!yyEw?I(4C&_y9@EldZurs$xWja9s$*?Awcgi(Jh zeq?pn_#})DeLd|&L_J`ynYJU{iY5cU`WA;_UmwycZ$D~(Y&U{Vpro$IzLcG_Tnx&M zjdJjeHl}F=)3n5VJ(T7+?k3f+Ylg;r=kxz;RNpM+9?{X_t3`E_A*xgt~lx%KRiqN}jVChdcpi^Z%pltAgTc!YBzIT!JLHC&AsB0KpxC z%LI3KonXP;9fG^Ny9Rfg!QEx>+5Edz`?j^aRbBUc=%;&I?$_r#=QNg1)*lNzPh;29 zba#LVoUkS~5eDv^ZYb7pe+-9s<2Y4Y48KTMbeDHFM!d}n95N-@oEb&2?x@-SfU|K?(-Ja*6hLdO3=cAa%HJ6n=>EQ<6C81&+Ww zixX1qdTx$oZ6F@=n>>?ii58!#8;Os(28z}8jjd4te;F?H-VRrJ4$K#9>E_Fe`%}yl zPU17R0LKp&b7LVdWNF1a&tm7LgjkNNi(^3imd2+b>07JZY+BJvr9dOzF<(5M5i6Pp`hQf@hvo4Hob@H5bj8xQ|_>v-v1>Zbq;{tH}3o_Bh2EKjm zeHJ$J3<$+pW-Hc;7XBp1w?FDP!x^tl5SEG7@0Cd_`V#{yt=mcU%4JdY>o^k59;an5 ztZVyn0THe@QLVv>hrbRPK<=OD3*!w>)6{m|$(~ zzBl84Xeid)>5#(QFs-F3B^b$)#{4(*G}ff7PY^}2KgiXXXiD-h*&Pjc&__={(kPu` z)9t<^BFrVTXS{C@8YVT^DO>@tUfxwt41#J}E$OX8P8l;j%vMfm@6HHo0{JjE`o!;L z{c;ss)N;9p^Ovu4SYkJmX-mj0P zb@4ki(na&pQ?hBZUVMh=3Lj8=tJ0ag<(ele1%KonoxNN9QCol=zgy30v>WL2pK^;X&ZiLqawhK(eC%4O_F5t70_Ge7G#{erE@M zvZ((S%3nPe!8Fwo$@O9A$S`9f(iozo7}>vfaQ3su(x>h~PCVB*jJv+21%#3Kje)NQ zs7hT>DtRXQfa&%`yvO7;00+kbuw@V$OoX>}dG`P83=HEY&Ift}hhI+k&GPa+fiQbK z7fmyVzQ=G9RE|+B?{h0(vLn^ksifxlVqK7ptQ|WqcVIpx!>6Gu<{Nzbn7Xo{Cl9^c?)cn~fP!8$pJ<&Kw;23UXT8{|+VcVc)?9wT zU{Z*_4!H7tIlFK}n%k>~s;3x9@w6*iBQsOniWh^QTnG}c_SZXslKb~ z6Eea;nnb8n=C%YPXw4Ru#N)!Tzza#KMS6EWLg<;~eBof=2no4uZ%Vt_G;1k+#w&ZC3cPpw8p_k! zMb%7pfJuxBf9KJ#K7^0=-P>bP>V9{BM|HECm1cM4Hh+*%fLh#hKUgGV{h)aF#|kxj zxSt=In0!eoa6j=}$Vd=%C|}LdB0*hb=~x_0F?dSzMfLqIF$XACCs9Mr&VXsqruGg* zO)SavAY+i@q`Hfpx%# zlEi>-&ijW+N(&LjV&iD`Sc`s64^YR|cg!_>!av?vVstx1ZUm&hZsHg2%TF6nT;Su) z*K5O(GuOm(Vyt`?)4A9Tx-|k-gfPP)bmZH76SP2wTq^&O{Cs^}==C|n5MJ>XrX8Bk zA2wJtznB}o>;C6g5KpJR8%v1SWmzeT8}y6!WCb5ixh;P+#XXwJcDbAONQSuw+=`fY zlsqfXu^Usnn0}guL5`8w@n^XRMd5mt_jEbY&*L2)`5YwR18O?*^^TZGl;4ue>Z^sn zv%j1XK6knb#h7l`n3KJ_g=9q?J*)0<*~}Ag9`a;lSBLacCG+6E4n;ckxUg^|);T)1 za)($S=fs@w4i%8#itVuydYCH`u~mi{dgdY6#VAaB>UTRC3D>gD=}$YNZkjc|#IpTx ziXIm??u7cz|9m1bf&rA8eW+XvPu>Q`>bAPnyA)$*AJYw6`#mCD!l%6Oo9P;IXkn?L zv%{|BcwzwDcI&_zw5Wbsx>MAz@ND&KcEt73qund3=x6b53GEI|*>zrJzvuio33Q6u z0; zN=BppIBbU>Nd1Sa*6^^B<7Tvlrl`LL|-{qdlq(8hG1`<9z z7zQ9-f5JKT{%E3e6lQo9zECq=q6C%46a4j?^@yhHOpNn%=vsCN?q@Nr$_7V;K{<&? zVnscxFFA@s0v`+5YcHLZeQ9sKb7!tNN931CzmVjbJv@Csb<@F!uAZj2>_VibL{4Xu z<4t6^ge-cTz9AvtpAM{ec>bSsKZQ|Ax8rlaacQVu48bw+tql&gdEQ0GABRSp#X}SS zJ$D!xzH7s0ozdG#^j{rq+Xkn{nyc#16JM{-Hi@2fPwQuzR)5LHzOB|bLp`bke##B1a?>_5Nc$a} zIym)7q$3%=rFGQRYg*JDS19ob@3S-!eu(khZ9TKbF_KqY-7O8fz_-PUx!iHD#9{{- z$lgM?b)bqatER_`dNFxqWyD)r2NtQ%W?!6qEyCT$I%}-Ygi{q3o5gkM{F*lc_1l!| z=e2t%cuF>c$h+N&EHzIi#fq%+My8N>=?^yhXZYF-545cn-`k#F1UWG-%I zv{x}Me(01I@IP952|#i(&%TG+8E+%%s%dJ+{ROCgl969oj~009hO1idj8bxVV$yNqKFfOx;Dpd7JBoxcrXZZ2@%%ahTOJaIrO--NHOci^sUfxfyEqB`lL5{)Mi!u0*91Jun+jv4i3v#Cd_?y|>CAHY?$dxfCUvAA`qQ zcZolqxKJ@PU(ZXLX7USEJB(dwCb6ls-W&G$;Ux%(9mtmo$lPnJuV3*kaM1%py4#USE9+{hVe(|y?1M(salFEw z=@eaU`}G#%=*p!Xy@rJ}8vF8t_Q}JFoK>6i@}_!m=F;EqL}3Ytq^WGAbzbs4itKA- zGoR`?V*{$AB&;jkxF2zi2k0X%ZSw4rVyt}1o8|ALh?$juEs3wCC@Z=Oa|VkOMI&w8 zY!4M;(e?R776z^!+FiO7^1w+cm7!K_60qxJ^Q$BZR|d;#0qz(ObuY3x~r? zoL);$!75sr!v+WlQ%dxDH}I{ddvK(?eZPh0zRL?+Be{tJ|HFE1F8*Ol52XkQQBpO1 z=qLzy&NI{!3pNXy8{w+`T?2g|p5|=kaxZ;|rT+Q2_)Gal6-gqhM9!VzZyNDtfVsez zy{QX8$qaro&dv;E3PLiwjFLk)Em>ep7oE4hR?S{dU>w({fEg3i867Sb8bV= z9>Ne^g!2}p4%>XOSWMx+Og7<8!Sc-Y(L$U!?%@!efgB#xo*MLN*9Qf;pN3yqLzbhC z@(Qcrr$JO%`6=znP}bns1Pwn=!~HAr_?A12Byw;>uMAgFnXzKB0GMQ_$N4LvR&Wc6 zU2Y7rfo^w&Gr$EPu?ZS2ft`GLH`az6YuGB#Z$pU>@5sL@mr-Iip(LHwMKW_Ft)Rag z4bQXdm@;f36fc$7(xxK6`0)l!;ePg&A^WF3s(*kl$U2oK*?(y(IyzZmkFPF`N;>gb zFIJSUD{As7E>&m9r0;{1M_%W;El3X*V)aPi4Y4ly z{4uAX@^Xl$RYK%ey>$p=#m)5&NAxi|RHnt7V!*$|*N5eMh^Go?U)iau)2qS>GDbX7 zlyQsKZ|iS07b*^S5SJ&1 z5tVS@c}m*6^sgS~aC>i0A1tZei`lOc@A_jJzB7i}V^5X4A5&!-(J|y0j_ztt1(02oJzzBBV^)b{hMc9?#jWAbTHt5$gdKd=?f zk&bgr!_A{L`>!$!g`jOxV-&%1Pzed`pizSSZ!8Y6C^RPOI6F6TpUc2=*$SFgs-bQz z<&~21Tgu8-{M#w@A{J8hlykKk$`yOQS1#(1Gpo_zvHGqB}1^M9J1$Hrq}vm zicZCG9dJ=Lhaw&0@#jb*Fb!-T3Y8~2YPRtTK~ai*FL4^&Yp_?AD`b>5V!k5kVD^po z2y*S0X{IqQBL)ZGY)oVw)))CcPE8EYyGc1lVb?Ig@ZM7?8gIM2ka-%-3O|*}-F|MM z5k`#x`BpjdS`TVLY@wPqe%#CpH|_V6iC`raMZ+L6|^wRr*e>dOSPVAfjyAhywL8;=R<3&+gpw+NlDIegn_T+46__8l%-(pCI) zR34w$a0F!Xjf$vfBgJgUr~GzQrJ*d7g7|atGGX?Il}_>$=gx)CbY+yGNyhj4zm}=C z)P=0UsVnN!J4(5&`Zgui0OM1oHg5cnG4Im7zj^kw(D~5rmdfH3I2BBoT+thX9|MgM z3_DlcP{*_gDA<@sS6sG4ZOAvjob3%*?AGk4<$*vTvSYL>mBo{5Aqg}1M_V%?oVaQ~r5Fn@w9Z?Cfa>wyqehY(C9i@_0j z5o9|}zvQbTI^St2$Nm&)+&80b#`!kF1^W!QY7j~W&RL|3OIz?aN z^7YeKO!7jh>^LPDUW+aj?Y+60*po9l1-EnSMfO)%t@8?}e~$?K?w3jO&kmQv0TrfV z?o{EgWqou{_I+C~e(%_ZFE1Akr-Kh%>Y{#p=i+wi7FKUXz~7sh_z>ElNu#Zg(JIsC z^f}AAF(fI))MdMY{rQu0edBOg+1uY?cZrF+ z-qF7us06O%2Jil+il_Fu$J-$ouW;(QT`{cH2pc+bIJJ4>IDz8jF6x!4Yn#vwHVWiajEY-*^Pi}@+-%f6BVI5#yh6-G z;&oiYeq-ALx-{B=Csv5Ll9>3_vYDH<{H~QlG5>Kr^5OKt5aO0cLYBGquihg_Cmf>|;DYET?=J5^L_tSXnk#8rJRB?}aXxXThQB9eya~kLesWohrGi}M(Zqj$##5e@{;9;9w zXXn!vk330(_gWJ^^TwekcGYT=vI^`g$4bmm#(w4Uxi|jmbzlE_E0V=Q8b$Ak-e%=r z@Q>s3m{iEWRy5j=CoN)JoX@i()Sq2R_>#BKciDJYRmto9)b5P^(Bp;uhm4v;wg8%T z6W}W0vtqVS7UkbN8`6C4@vp4GHWO0iOx5ICtL^dYy)XG^k`Iu1i_DH=a|1!+KvC}I zlDW?&-K1*X4q9oDyapdn3iZeqD5Lf?* zK?nL8eI|JZqsC;nfC4=W*!*6H$>H5rQ^E|Ii-?3i>f1|q+%t#CD+a>&Sj}*zMXM)< zLcBtIi5<(i?a7NzX#@uJl?aLu^B&`+?PcJwgKzHx(&(`oC#63EtY@q_)$k>!x%KND z8?GhmNfl~qNRSM{nsrx{P2wV66k++S3RcOatnd<0A#V5nHKxNwH>$+`j@cpgJAn3= zjWr*_$H+^tt*XG%{_e~gaA+O%h2Z)Sw z8*Ce>)*VcWh0URnh*XYrA7z&;XD0|wRQgIZ)Oj&_lmY~ zM$*4J+862Pb}#TtgjV03&y;328ZPXX5)-aUuA;AhH(#K(2*@m-@bz9C4U+1?L!XtI zXQc1gg4NIA)-U0e_`Y|d+VYKlh?E<>*VfO822LPhz+>j+?{%i;om?j;;9wT4_eaW4 zSmuawK8VN1eJcEPLh%XuR9d3@{#at{?;4rXZ?H|{G%{y(i9;P? zv38hSo2oj{^m?klR|}0c{W4_T-S2k#+5o)uQ2LGC_!Tph2*X5L9(FCB!1ykZ9-uv4 z8<>?0^w<<3=*U`jLDnBCpiXxxiR2{OQXEOF1^^K*I;$c5SB<_Y8Sc`ETiFKkyi%`_ z%^H^c+Ar%B$$VUKS4D1kcNnn(G&qPA#X8>~whTMp` zcQ(5gf{PUydL6!tIZ<>-RG7z^r8dwHUgWm_HWzUA`m$I!QLS~aZnA@b;_IaZpNl-o z&Jc@A80`oyNw>qrjX}X=1&#Arh`yr@hT_;(R|SBh8PUgNOynBuGKYvuKExv+BH2_R z;CWFZiK`v{q8-Anw#o1*`x?GNUTqHENFQ5Z*X2ED)Uo#MK4ddSMn-w(OfEH8P?@eq z{YPb#mTcDdfWTw5*nr(%$DtQWNL~s709QsTyVHFtR%19*l}#aNiMf|S@#7E2dASx( z()s_g-JjM`yD|dnBxiK|k`bFpEHv(NgV{Yg=KMQ6n5vYv`uHQB+rPg8LUuJiaTMW) z4fIT?9v<4?Fz?6?w6d;Y@_nEm$MFw@{Z`qU*;^-&KHGNifMj?iT6BbOMF#|I`=j>n zuh3(TEHL$PbZ6HlV^i*|!S%_h4(*&fOnNsB3Sae^n0Um;W}FoLQb)>$nuy(5EKw3j z_8B|B`vC*>B0$Xy9XH3CNxu=Wd>xigOHNl(5L+uiFfaa4trVk+KZe4tcKuc0BfY?L zWa&B%KGlYSmRR&1GcHoxhOjp#%%yrOB26=&T_6PgA76j~qmEd48Jtc^$@ptkSZxey zNA$I;ztM-{=YwFZHR4))wbTWu5uzgP+NL?9M{}Jv2~YqE2bJpR5|rX)n5JfwcC@Fh zI0vXU`(ih(KW@G>4If*%`}^l@f7Br(RM9Q7yzhlo)Gc0N{?qvXulWCUwE!Y9+*ulx&51l=FeVxr8a%njEb{ku zBqXGF?~wFgo79z+aW21!ihdLALD2q^3qkP@4mKpwS~fN9w*67*R%oSS_|wpE>nYV=TimX-h%ybrSq!yb1&^Nz@)qxtpp8LA$Hl zS`6;N?yu}8F*2};7C`u$k0?k%h08ZXh0NB@#(4o|_jF+iYcJVOQvhGs!k$aJyAIr8 zz8X@bw}TfKuNNmQPpeJroBCu|ZzcI)1^imOXEziB(BTvyG#}=xB}JO>=25Y|k~1&^ zB44w(nE_oq`T0Ev9YA_s=iZ=>WLl2C25H^|wfHotAACi?PZx91FKeK;otQUOlAVsH z(-K(r+V7UB!PA21H^QN|i)Vr3?M#nS^_lOA^A__^#|>bPMjYc==uLgMp8?AL&+(h) z#W?T)w4FVGj_)EeL*L?;sdJZEt1dIj!G;S!@9*)$4(3KY5!)=IcKU(^$J0 zOvbPhS?Mf;}rOjUwy9 z*GDfDm^U<45=@EGJKTka_{#FhTAgEgcLKC$C3zj3@{y-smRovISX6hCR{73B0KxOJ zaDlzl)@TYkJNG;kk%6artWECyJ8JB$;jX9goHv#?z$ry`#X#IL_)f@58t71H5kh+T zK?osQHRm)bQcAeTw2;VO+;XG0)8RD8^kds_^(BHwfM4^mps}L~lpC>Y?Feb~7bHTY}DlL=7pylnW`;{?UAPK9j z$L#XU*4Eg18d1q;LgpVbV>#_X};A)Q=(Zaw zWVYUzlEHQ%8luar(b6b`M)z!l7*(Y|O4oAaI>^lqu4ccL>_47B{bQi_4C*)9nD+V&Ub+1z}-oE7X{#e=ApU5%?0SS6>ZW$ z+uZ8iaGhpPXDfNeqrc*1aGVEoB?|N~=V0_k-3I!^RGYizokIoif#^+aQFiRLwm!xq zcsl0dawWL{kY_Ke#w3e7)zxY9G2Y>D4`kz8{G48o7Ka!^R+4 zBkDXK*ikyO$zFOV&lG;DuYMW4qd-AJ&j&2weaa)mUJUmhmHM0Zq8S}>9}jHWs{ARS zDZa~*plfMYhJdnaV&I!fbH&D^mAw6&%ixnfwdsoMo4W)g$Xmu4nUKglN#^OL+&-f2viyYVJC%f4m4mF!Bf@^6i; zy(YGI>Y7gIx!-v5CnE3nV00bHl)*Z&U z++%Fjq_Q>|v+KM%-Y|pp685MU)nkAAS8G5Z83lsiHct^<=;}n)g%oXRuY)4l&{>O& z-a2{%=^*mH?IlI!9re~879iAz@LFOY3jwY?k7dt2Rye!YK6i1?KY%hxk(jANGP9;aF>! z+x_P(E1#rpJ?VJ6!x)x@(vBjQefO6RV5-gN}lyMA#z22$JTA?zE zxqjVwYuEpnrL|c}ms4*R0%(sg8ZS#(w51OVpnFIfQOL_)4;A5HxaA>=NK(f5m_TUp8qQCd(MmCTo@T1i((g!&}efq3l{J|=@mFFzv zZl?9cR9G}>);pH|IFx9^wg0Krf$TTE?1AidN24%bkGR9TK;4T#1fe#9m{vdejF&3c zYO`LS9HR*S#H%gLSlagRdSUGsc&b4N6tNp-q?P>N&rL2a=YY7d_Gh64&60DZ13*EIU*}2YRtX?ZIg#&c`iV^NTr>5c@ zx4o@8i|=Y!rQa5RV}Z+irRD^=Jv$d7?$2t9|K50|LwETd&&Q^RhlCF9YBYVhi#iC` z_wKaMx|G*r_Gn+eT?{Bqc;+AhXUPpV^QxcSQrmmrJ5{G{b3}P75RMYu91toXueMyd)+5wm9@l zcnay~Q0nW63qGRjFab8KN0S)-SbTh#RPS@uF8Lim-MJ#o?5-wS0wu;Orfb+K+R?px zqJQ_VQ1T+D{`_;l&H=ZlAvq)h#8f$0LbgaW@P2=nQzJj8DeH&0aLTraHTl5R#zli- zwV3&Y^+0%X2?DW-X3bygu@{5Ke*-g!ewP9^B`>M=wno|)5>(X$&yUP^(L4DTD~2`K z*iBudU+6!*1iS9}6!9qTLG?uNDO>}=49CHq5RHl(7~w9cV5sUZe<#DmA0Daek5L?0 z{GB7seLj~XFI~9~9UTTNgMsFk{wZ!+N_r&EX|)Dv48#Y%bOtTuzrFN4DHJ&D>B(LC zVcMS6(v^M!$S)%od)I=Q4JllJDy0g1;{1>o*pKwJHVCEb_zC&LcTCWyC5`poeeG>}vo*U6OFrg6nV5s?AS0xrm{su_2E>hpb;3U$$1N#B`XSdn*2;*gwyI5R1Bzf82 z&wiV;>PwIn-s{m}-KD+3(V+)fr7!KH8&APfY<=l>(wAa9$Q}wCrAc-UpopI5vg`5X zOIAoWQg@dC%9A2v?)QF~J9LlwtN!bwuyV*0y0tqCP^t27Vfr6fuwwvpkF`;ug*T4X z`ghU}DD*4~#g(t|O%pi0bmi(p_CLc5#W3id5IWphO0_e!%!grJN*vVhi0UD^$2g2Q zp#-?CtJ$Tfi17ZQp~U{7-)huR2hdDRcH6`yov@lY^Gnp`OGn}u<3_x5Ji*%MN_?Va zZC;1xo`5d!g}3l~L!IAiEm8lw!CtirsBDZ;Y4)judNS8CgJRc&>OIY@Nl4U4(mL^- zz-tmP;kw#a+`hrhoV7Ins4U^(LvJBY3Y+ZZ7dxID8@~qj`bLc-c9SOZvkm1I=2P&> zX1MRm>IKiet-b9{$SI3$Bs#PoT>iIsz}6}I;CcSlM^lgN`UOvHY0*G7CgW9qyA^U$ zEX+zU^^r8R&;fR89MWmG&C|oSN{X35b8_Eq?QHivReFGH8n{euTWP-Iy`8EQD~)AALySqjlR2*NmKt$`DkCb+oK_TMX z_q>)@P#|DHV(VYl{e!)eU{cBM2ETUYM*KL{|8McEc?*DOa!u6O=HQ&4HD`cVZ2}UA z&4sFYGJDq~#SnHgX73Rw#Rl1c_T7y?O3n>*#Pj4|>&qm_U4dlv4knhr8#8ACKR3zh zAzi@(I2R^+vQffLe))v%Y2S%P!{|NqC4NS-EQap7)kOB3K|W32!Ru zYdMdaUFCgDExQ+UiSmsnGkf~FjC^ED*>Rjr6Ls+ifRFx&a@5&-CHfGhPBKuV&a|t( z7O%18#+qGXYz4=RDL0(8UO)eQ`0c6}xAR>F;}H^=bAk6$=*-LQr>jI&>2G~}0qana13Xyc;Vb=CuG1|J|%G$yNEl%rd=SBV~Z=L=A= zc#*vwga>AieryP^bzsOb-_s~%Z5%7tIE*`t5ox}0ln3&=`2^sMSV;t#hU8&TRetLsh(-kBUH z2nO8MdvQ(Oc5Z<@O1qo?Kxb-=@{<|wXpc)Dn=0;-9`wJ1b1A0#%eJ2|IpP(b@QLRI z2LTdTq!hzr})+f@@XmG z;>2tQ^N?_KvW_zB{JZ+?iXD~BLnLyDD{GY&0^nBtijm({! z{2S8$>%~3m|Die0-hVX-hq`@B286?9LwPE78-@mi1F58S#dX+sCKL8m#6HkbQ-?-T zp2y*cj^L&gRFztPKXjy(ip5b&ndipT$!FP$TV%`nxbvFnnf*vI^k^#le7)g)*0_0B z)d;PvNC7m@-NR6=tri(A$0o}c=oBgPMhO_WYp_`>tg%8Om%+z z**`ke6|YrDw5p&*(&}?eseX&*bJo`%pdhtbbKZkjjoD}2yXGORY)h+w=oAIk_fG5CUEc>v|vG$ty>1-W|XHykoW2dnA5(XT#vN-CR^o zSHGt;_ffEEnb8wn=oVj59&TrCj*g~Q?njSq(xp~Zmeho1} z6vA&axPir0Iivis$KfME=|-$qVt?`6`n8Mk;C3nE3&?!k-?b3>xT!1UW4s5i^r9`xv!^VsS3q0PV^sPyjyI$4gbVIDhAEN zu_KaZQCbwDcwDfTg;TU)3K1KOQx zu5s$09r?I>cdnwl8mOh+(eyk@cyL}IKK&f#y_7<71j=~!IBDu-$lOAcvNfbKR!b|o zNd${0GzGJtzG8a6z1u|3%f9N{{OvmVNArD$J7W9Op%6WZFF2r?$lsdvT1TX81?y_hem$kiK@v3@E8Fct?L zNg+Dga(xs(jOfKK_T7*h$OZGQz_^H1Va{H}rgcwVQrX28-FVdFT(747b~C(iU_6~> zoV&G6B2!HK3mzawJVyOc*#Bvt{@{rmgfaMrAC_OT5pTG1oKx^(BHI%vA<{)hnp$~l zE+KcH=eG_ITn@V!hSW;7EoKCQ=Jf9b?~funr(%b*YyPZHLM(&G!i!}Vi_aN}SmeoGrghzTJ8PWrPv^6}PLN*`5U}M7vtJZjcw;P~?=HxC zKsa|6>Vx&3Bm5;a6Bzel{xQ@aV@$!F^M-_P(S!E>rB=%p4cE3kbXX!@h;`4F>T!b59wX;3p`BJSCpPe)iaAppYyu1^&YGvoBcaRa zSqnSN8ht$gR;Z=ikV7&>CpZT>DO`UiwAo^KfU)dY`^`gwfHy%c^8J2iS0JRF1rwBd zcmOS=85v~(%RdBR6n|q2X0mOD=Jxj9z&6Nt8zL^~K|JynSR$vh9b$1?9gw41Gxzm6 zX+iO6aID&l=XtGf9-R5KhsH;N3_L}!XNEjdZ(q>z$vAB&_4?xYX^jdTlr9OCKw)8^ zr(AR4tMIVG^VOK!sDutUtS9I5U&^E0hqYKfx9<)jLPOjxdx=kl6TFttVsG8cK!R?5 z{u6q`T>}zz*JWRVi`~Y%_th>B6g&HU2Qb_>#Cu@$mpJdLN}4Nzh$}hKg@w8n-9bf@ zub3}cf!+UlsZNbkgcaxm8h|q)Q%culC&q#pLB=UX3Z%{Yb>lX16*F@*K1Kp6FcQXK}ycU+07D?!bpL$xL+uaE2hn zh{m%YC9`Yl4gCiSthq};D%unTysw_<t>23R1k%BxaUL(%qw^G?*npM;pDr5pr zmtU$&eFZhT_E6jOF5)YIIyj`s>BqlgNuLHuKBdX8o_I8SMt0@|#GsiF9Vc8t=)!!{ z`4?CqU1kW_RnIWz<1&Lw1>w`1FVNE8r$k8h0JjZ5tNl6(uV4K<7DLR&Kox+}cKM9M ziO3aFt1`Y+@7%%(6y|AY_>xg_(+UkPgdRRzgL+6E;tk9=0 zb7Aj&E;00?3$d)2o{3o>p1*i;)B1JgXUq2)2yQhxIlSO|edy!tdf?Z*%jJ)xV7gYo zx@qe-#`avY8hJBa-f}*ltL!yo-o6#y{s-ppvi2yEo^b82tBeFf%#j}ZjN7_&LkwEy zOZ^gP&mM#$y~+fiv9T$_qF+n&(2llxH-Lqo&_~?~`trWeW92x3Y=d7}4>Rczgn8?yhIccXEdguv z+VI$kxc1y0ArsU9b4bdMOWknN_8+eco(w)Yo-?}#d&~Lkcg}c^(65j6Kcg1EJGrUCeQSY4 z+bt*;@Brf!mdhcTV|Pumwji&9%7U-Vedtd%v5b$$4iv(yQ%&;b5tQBQ#aeIF0~Beo2q6OpDJA!J^Ook2;~ZQB7`tz- zTrgN{wqF{PpR^r^kFBGVUD|wdb>+VFAp*tZPe!S{aW`(obYE|3Q*?d)<5v6 z$tgXq#(op;r#lEo`TOCCGZad4S&GD>|3L00?Q}|d291Qu@xI_olP%}Ge2NzS-L|O) z*%*hoqaNrCz4B)8HTZi|Tx;(m`$CbxG1Kk23rKHzk{v*Vn6=iJCL^%tzzV^`1y+Y$ zvBBCzr!&`s+yMESR&k~mt>#672I9j}@GCTd$adf_(%G)LKoe48A}CDuiX2EgRz;ywC3Tp!cLNjoHv| zyvIR?11jt-HF2y~hB>vF`ACiqN$xPJk*G`iqPJq-j}}jxAup|Rc)jRYVE1s#!?=f*5O0Gp61MB?tA?wZlNrX}a zB79KS`r29#^zKlJ{(Sf-Ye6U652%Xb1J81?4FjeExdD`?d)1xj~76`{egZ1Kh>!3V;@jQ$o;B8>6)anuTf$) zXS3k7WJAop_1<}=+dSjaB>(a(yxq$U6tSBu!=WFgX!pSB z6sU}e>tcQ{8(q2Jx34-%@tGxqWcukwqObg8pP_o65;kf-S^?PgJJ_3~@qDesnL52u zB>KVXwlm5m+2EeakmO;L{OI-jTagj3{?oo?ih)Yg?xn`)+{n%~-sp@qE$EQQ?0*1S zK&8J@r_)XHcS}%uNoB%ue&P9@LOvhiLChV=y{+c$)DaxbW>X1CJUvfG7PaFPG zm*RcGThz5+zxFR@={rAlO{$kX8{T}=CO?Ld&mQ@0uY|NexLxR@-hTP*bOxmFyf3-! zpzJ1cP7`ru)ydv)&Re56oZobw4Gm$3^Kb9g`h}qMn}v1-{`PLwdXME9cCNwq&^F=L zZ-wWPk5vi>4(q!an}uV1TJL}l7s=nd_%VK;)+g&+tK|F?^|{S&kbVUHw1_@lw?lG( zzFdgi5&6Q=>16#&9^yhWzcdkRSpY>9Bu>?PxzSny94#|yP z+3gaaEL{C*_sWi?5@#3nj3f_>>CgNP+D}p3*2+0gFdv?z-&4}H-iMs}wd6^;!bswe zr@sm{7oCmFp&NV`u+Oxee#4@)BWX~qjS97^v{)FeFgcc`z_BeIygl7-a&k@YIAK*nM zb-pQmJE{5Q%;3K1WDX}&$L;u=Kp&kX?`PI)eHgr3%e?j}E*nn#8RW5_KADIAv$!Xx zb05y+ym0D0pZRx%e#yb#PU4BBAHAHL!TsqJOV5wK17A+SV-It7Kzy_B$bSxbt`&bQ znK+&0!XWosAYJIik9GL*S4jBMUad!Q?n>tUD&^f)s=jybbL`NkReb+3^@}C1gNn;) z$Wt7AdW|^{hMx)Kb@?XYS@ItbKhM$6v(c;Nz6j%9hyoWAPqp&sL?M4ppY(Iz`LWxp z{;;X^)Adr}2Ikm2>iJes_GdBAs+kAF>H8r49=$;5E0*2W%&$L)|C0K|7U8d7?-pl0 z{7!_=Nxb)d)IDZL@b5DI{!SfM^Df9Ej}h>rP5H9s!;(`%`aZzBrh<3WG~RV7+}m&N zlHIY?zrRvAB1`)nd>6&}qwvdDEjf!h(T+a_%!xsrZ*knCN2%un@Ti=3+y>^v1@iY9 z^Xfe3jpX}Q`eh1p$EFEOc~3q@9_yL!6Eq*4M~LT!=C?gX{T_n1D|jb-0Z*f;cOw52 zYB?{5zD>j4Sp83N@~Q6&Obgh43n7v-Ccq-89icn;RiVA{yLPSjhR8#~NR3z~O(G<|M@D*MWNXZt= zk=ZytzdLKiKYN{X&pr3-y`TN;efEoG-TwRXq~m|WTE6p+G|!Z2+%{iw=~2mng~fr% z&R@KnWozBIW5svoJ@!j3acRD{>(1D;EYV;tM{x{8zdi-zs*kc zZFxxYa)IWjmS~;NNzI46EBPPvtw;7aD7jAGH{FXhUe5U&_iDZddtWQp_>qtFJPAMU z!A_eF==rISG(IXES%aLCt8BleNmk9)`@<`3{WX6jRpUYU<0;+?0*@i9H1FgbCEFz* z07s+1xhHlzBfpyj{1JdX0*IS7g_1|G$Ahw?bpdKkr4~f z2ffOjk||kwJ`1~DIHva}Xq{;X?slO6P2%^DO1&2?JD9PYYs6+r_j$a}_Y1*=7yCN! zPZahlOP1^)KbgF8>>%8j-+4dspvIF}7t1+bu8C;9 z*T&I%+NT;md=LIOi`^f_ehrFC(*=87$KRFG-xPwkKDM7XXudV}+sAp`%8Ns9|9SnL zxb7w1HSLI>dE}94@O>)r)Q-F`pZt?cyvGSA)>ZVaiPv*Cr^dr|F4GhH^n6z`f0y3- z7~J@*ksOP@2Zciu%{~jow^rmPJuf8w^Y$=Czh>lzNA#ZAth&L(68E$5>ojoQO8&Gu z?9zM%`(EW-)_l!}lb?nbX`HTnVVzeUV&c(n&j*rO3nl$?^j-*dEK^)q1L0SXGJ9UK zK3B2?-kuL1xGP@Fdi3g!Uz@=DzkuIH;_eUfVPE__0Ni!qJAr%eNOOdBiEAb6;QuV@ z5?|TVjEApBk{5qvzpLbvJmSQi{Pi>Ol1!dl0j{z_^t|T^^arosAQ!=()&R-kshV#9 zk7;~&r|KhX5B|zTk9pJ`i#e~yPR(Dk>kjz9u6vroOWhdTeEu=>%fajKcKoP5GZVp2 zU-JJK;5?J_-$3tC*x?~?7^w4`)x2*hPOY)vEen4i0;m4U3uY(ya^rlXw`$yJot}3F z2MgeVyRla>Ja?G$hPRTa#$X3Qnh<6{}zmli(x?_*vNXf3~=Y=2b`k^2ElZ;(DlTQaF>G?bOaXxwD zQR?-SEs~YgXP4lwYQ>B3QhqjLbPm(hrFGNbk#DHG7Gakf>XUQu)^yI-L~&-l&blSM z7lwYN*s%|IxB>qs@Lr?t8`f!fXA?Ltf*&l^$!00vErOSuqt8_3yW9Ce>pMF&e=$Yl zZmc`NJu8@dRyMpb8=eXPmko*oGmdz_M*J0_&qeB#3UK~2cyxqm+ydU3gnhE2wSI_j zVItB06Y$%c{I~^P@q_2OGk+yRvSh2?-_H3XW=ZA}hX*9yENWL zJ(9AMGX7{u-H`%+z6D-=xW_(0o#?wl z^T8n+k0Rfj=e6HR-g{l~X?Ed{dd0VK17}m=k9y+GH%BrI9taS>oA20fE%kO2wD+nyU(#-GkC&cp9cK?8hl>P`;El;a>b{4f$!vT-h6PK3eLx1-`()(t31!5 zKF^|#cH=xX@sioPPnfpY-7!(~U5VR@=?`8vIwgHo z=b2^Td!|e4o0I1bfs1{dFM+!E0qTu-@LV$k{kR{@h37Nj&!>q0v)t2?*uN)r{(kV> z8hd+k-}eRg55T|AW4~;khZ6UBd^Z?oHLj{#CYy2CGcnldRfm%dczkFQIC$G zKA(dhegX%p!R_75+xK?|e7p@m#inaN#{|g)?#H=&Czv=p!#(pBb~p}?)*^$^@3in| zYQX6a)YCVJhZ)@e9>ZP-=u5;>|1{#~Y@Q#+eod(h|Dn3i?1g7Du+u{PoJ5`4m-UhO z_Z)V;ApaRJ;(Y*hWVP;@Rt9-?L7vt>4{yx^A1#QJMC>t@dO3o=#-GG%gw~lL&N&x9 z-OKZ2_%oR2^&2HeW4}P2TZtNn;MbOlW3wKf9Yeev#Sf14lB@80D{!)p_+LvNA4>f@ z7JULWHuL%ZSMW_1eA1kHzl-vec?A5l72eGnc+i)8F$aH?6EA+~T~FRhfQQF0@0cSw zh&nJ7`>sLHf$&ra@!e?muh2gl9P9u$v35K&{u=*ZXa7cUGEw=)L}Bj}*g2g1Hx9kJ z(Z8f`VMT+}Jnkp;#8cz;Aw(*0&&zYji)hlEhD@0{?r`{|bSp+fkn_vio=Vvy?m+gMD1g z8(*z2LT@knX9m4SkWXsizfJT#VzFaC@Ysp`5{5sHa6fm)j`{koiGiPIfP=a4X?%|M z{SjRJ6TRKHN&Wy&y$ByQL!Zmk(-Fk~cIw+myN^tN;Tie~PW92O0KPLzb~lrWw{MtF zpdN`=AKbJ@-$e4_EaLPW=X7LhT_O4n*`oR7)V+RA&G$#2be{X;_x+r=KYfOt;NPD< zsAHUDC*4C$3-EOff0wiV8ufD-_nuwgw+>#nuwyiOUZmd}!@5fD89g}H6!c8VkQ@)s zyhdDaq#p6(KJbN0^EWdkbI4~K;G1vhE3OBB1?25&@R7$k95EXALa!wDZNy)1a()9| zE!ou^0Jj;OFN*pv0bI@mC&#gqKmNEy-suQWq{E+mh_f@qKW`or6?nVSOwF$RARBv1Vdzs{$A7{-0zd+J4h`0}Rwe*4}o>4BaVd=hru!Fhj&=bq+!C%{(~zjJk3 zE*T2Hj3ZuBz{?x-osJ2=W+%9gME^qWBYz|B<`Z`rd~YKD^no`gQAgDhf3Jc+Kjv?t z|3uD7U&XAZez-zBc+o$YMBb}|FRrYVya)Wmkw4P0V8)C#Y~5v(!iw``&7xl zmjD0zM4E3x-P4czb|m$>C->yvxNlT}i&?}+1pFCKeyTyQEl!Q+($7gI56|Ns>jZbP z`n|y_BM*CV|F-MwXmDZwzOxKJ{FI|{823!OZ`qIa-`o5O&$WkVU*ue#*yjV{F~!kaJ>M$+(Vu}3IEQ)js^5#*U`sW1Ml~y z&Z@@WZ-VbO;A$SY%cDL?=rddEtPb4&YQcv^U#&fO*a;4! zsQ*IZi#_4*i

b9O{w&^2BYT~@HLwFDdoJ) z!SfmX_9VY!d@!z%iLQ&W1oA$u@k?2>eT!v@VEW{3+K64 z*uOt|6W3R6jx+a(X!m-1%;!kr9yj=q~QPj6X zh#u!WqkFEM2hDE$+ptXYk79TG_lF|g@68gI#63H>H+DTA2>ycbPb_w>F`B=C-nXa+ho}!=?xL=!AP+xC9X>mU^N>f};rF|E|4Hzg z%>OARv41*tolM*vV846OzY=>-!EQm^^J>9YCHZa_dE`0fr{TX^>bJ|}{a)ZEjJRwi z{}@k~#v#-hPW)QM_lDZPGtggpiF&Jm^E#-1=20g_;XI6%GZnD!+&k8 z@_*h72JjA(S5Ll41K?TyZ~qSA_e1ZyTc|v`kf#rWxvF2?IQr-+a69!gGo_%f2D{QW z(A!I#U*u^shmbdBVDHES`%94T?#F&<0QrLvc(CeW%fWLV@Zvq}Oc}hIe%NICZ%&cF zdD@D;7qs!x&%V`v3Xz9-;qC(X@qX;KHNc}r(fgPq`%U?+YM$0LUryP*aAEgy3VE^Q zPU>y~)c^hT-+tizrm`D!h@Vuy-uvKt^dGn8_m=wllYC6V6aK%dx-B%pz08-5Q$I`c zl}|4Euc}jU2R!69JarPk2GBgyzL1J>X5aw(%YE!_Nfi-LdM&AMIJnJL(X_zS=V7`-*unK~?Zk z_Rv&l->;;B&Fq)0S3muAHv;8voM4jblO(yL411U6bwPI}jy*LyHFux&&V$O|!FVEh zdr+QX14;k@tNmZg!1n)tJ?sDf`+@jA5WffF??C(;i2nldKV0(l|8xEW023Af z!T-4n0Kxyc3jkIBfQZY;ghe;6I@ZjCO?V*@s+NQXFp+QwmXVchIT!2{4_lMxP*t(6 zHdlb)akKe!bSK)n@`PxH`%n zv<+*@7qM(+HB?dCMU@5^NXRi`3B2dn#dIyq2UL_^f}V+8;YyJz9ih!g5&_uwAwZ|w z)m1{z5Yli>N4v}~&zn_G&MU&i+v9x;J_^u382)*J{y6IV&n%Zz%WO1~wIzMu50@nQ z;$DKTf~1vt!{)c(&}sx^`NMmRb%asX(__Hs;W)U$=Zb@Fbsj=vItvD`LQT zV418sSFh|WyVmes*D#i;GBaz5%M*b-CP>Qzk`IQqdfEom0 z2CTehEPQu9%BaB|qP&1ks5Yl22YgB|%^r&4;ZS4|b#xAUY?b>|au|?W^3Od7EN*pJ z10^?KI;7!Hr7$3~jk@XQ92Ka-9XHSg)Xd=m+q`WH-i!|riqJUOZ{X5JuYCZ3GmWK+ z&hU#IC0}|c@tV;Qps(e8)b`k`bZWHhPxF`fepK%x;X50bCH9T;=M!pR-t})^^jCf14@4~mr`>hWt5(7xB1lUE%F#Y}sfnKUa?+jtQJX!I24&sJ(GfDW;09w}E3+q18YG(voF zq=8JUJ9@6NP?u?%(u2L&ivzR`aGm^0a(d{zYAk~n8?H9~Wxnq)feYgJN?>d)S~2t` z6LnDi8eD2$SjRG8a%V5uo(h($7lYC(6wSP}oRm2mNj#gnd#p#=aI)<0cn5A=@N`Rc zJ>6(sPA04F7_V4<9M28i?3+-Y@~=&^vCjZr$VT#l?ztcG@7oE0QA zQ|zsq3(>3SalpV_B0LXphHV_L7N>IECK1jl4k*ho*`DvsG=}|#FxDD$FO_;{deEnY zdQ@rDPxYX96B<5^738~@Dp2c$zp@0eEo3D{O8z?-5}f>0a^~$I)z3SvM%>o1ZjoV1 za_Bt@e@;M#@MN^(_fj|nDo6nu-SrVuKg-dvQ~i0%@}$qKM|o#m!z z;stG1aD8zJ<2Vui?$)M~i5JS-E9@p#2kro7`a(RA8~1HGGn>3&du^$&PHz^-8%gIT z!RyPjZ|P^TCG}8nYAUtn_~8D-;x=fbP? zsSor&xyvzC2SLqrWuONcL5+0~i$b#W7=L3&x4M?xliD5-F{b4U8Af#{QDj=1UV?X3 zKxuqfFANaw!65^s%!=w@=a`-!d6u@Dwqg3hRy{bmMqBOZ8Ap|u;@7^E?s0i)u2HVF z@%aPt8mu! zk5U?J5i@dhh?@#6i50(0jzvean3|b2NYilHFb|xhMCL+EDWlaZ=qT3&@`N8ljX;?e z0zrI~7x7XQk`%!^g8>w!nL9uDaVXM|Eu-c;ocSjzcG0_*94ki8z zYCkIA?-!1{shI4b`FSaIYoXZ5F`0&rI-U{SmJQlx+x#Syr~NR_O$1ymnb>FCk4?)X zmLihI01R;4k@;)lsUedS<{%`lUl=sifM7fUn075!W;#=smg6Pa3s?9O%0Ids)5R0{ zCEu`&MXnoQCd32}CkGXtw`UImo#U1u3SqH3H!m(ME30`0z2Da78A>-ddxW(KA~TkY z;KhGpsya7jiCk`@ULub9FDfhIz!t|&MjW$duxM_8I%rC^q@!65I7dfqo;D`HN~tuz z^Uh=%WIE7B9_F+@Y6eG^(m3FFDg8GWUhE@!ZT2Dx>1!P~#CrZqTwN&KDvl{4^z8uU z=!Z}`K}juq58Pkcg0IaFBo8nY)^D_?dZ4PtM!Td^t3alzj(SF`$?7+yLPT*N_P(Qo z_S4EpK30$YavgecusB8&o0joR)*WZr@Qa-NGL|XodQLsF`YmTpu(Um^a~v`aveJ%* zm^i^(S=Z(t$y$N8AI8-eA5}JzV(i#yH;T+|{ZSA|!@fimgB$xuLn|4*189PkEAErRVRls`V+v^b`4MgymT1?--9^&I%3-0%aHeF!BmyRq@X9 zMM%UkNCPuT!5a*av?*Z3%4c$mYQDR8lFbCnN?C3MTbZa&;-`S5mafs|k8&byG-NO>p60)g`*9AoF)j+& zB2V9)gcH45qh?*s`^hl@(=kn5I}+8J-|1lVXT&pTxH=*zZar#307so?PK(jr$HQY- z`XS<86p5jF8Xd}chM@|IY~Y-ZV@3kZV6bHbB*@dN2hBqS9^_(NP?i=fS?*2DjluLm zlcIA~1g0>Y*EXRYRje42+6i#~>QyH@6EA<)Ir(L_&{21insr83{dK^aUtOCdi1QH7Bg!sX#un+Rq zfUz?}8Mztk{0=588BI_=mFxI`vR|_RE1}YUGu(%+OdEEQ&i-)O*(*!9F&(auC`Z*! z4h*VhQnSrTv!anVu$T&tV16D_aJ~8qXC6TWX_mJcJUNjMdUX6{(=D}OQZy*1lOp?) zw(=Qow&Wm#*-ZyFX)v#;v~%mdayQKa+-_2EColURY2qZ@g+(1-TD3I<$KM&SfJW!- z`b#ma}0QAhG=~OhiBG{&%rE}6Pk>bNJM?Q1q1QF}3F&_vpA%=VN z>I36@G%@hO|M3I>!H4k#00{~JNvCa)b50bo%_cJQL?bs)>T+KjN_E24*wF4#n#91* za~6B025OAE9}aEI={`1Ouq56U!WTBe4Q;MiEwH?xXvJoo(B>ufyp(0x3t}8;nCNNJ zY09OJ8yTi5!eLaG7EQ{-9xHlJHX2v8i&d)?No-shRh5}*IY})GLdI8#VS>I!#T41X z(tV?arl(9Ivbe^CCj&vzk|xO=*risbF;{(48CjUa?&{$fJuf7js%|KQGR}FGdtp*k z>Qff>rRt0(6LV(v-fOc=g&AmG5___+dYvZX(~==99c@vXS}d+)Ohlz^pY4n@VVL z-19Edfn!9S$%`8kwh+Qs8Nr5pWi>s`N|?m1xJ1<$wSRNwkb1Y10$MmyA@mJmZBuwWXqo7Sc_i<|ReNS!GPJVcN<#rCa2Su5-j0 zi^U8(5R663z6%yW)|t&pwu?fflohkRBLzOC!H}v+Y{YSNUhF#LTS?7RmYZDiZEk5g zE~AviMr%E+mfo~`?;|oNli4Iauw-h2#*a(WO0n3IW~#|oxv-SsPi`QG*=LiIT9pwh z85K0pV@~wCWlfY*Vpv((iW)AwjVqTGYgp(k&9&zlx$=>V8ZyT#mZv?er0)l%tu#|- zTNJZtYt`&5O-QP)g4<+D(zYup9mic{{2oii(#n>TqvZ!NN5e~r&91ZE1gNq~;X9bH zL}xg`6v7~A9rdAHWtB2v)f>F=W06+?00u}BJf!ut$(5*jrZYU$01Zk1g)IZ(%c-YV zB@)vzm)$Tn9&Q#(rMTqc)Ehm{Qrg$T=7-_06C$=VW%{b2SMlAzK z&wc2G8ZLX|ELzEoB}u{+YcATRhAB{NHPAzkN!4<;BQ?4rRE|U5j+Eu{m9e zDh!LzWvI!Vp;pGVej$s-(SuBeysV;5S3PUNN~glqov%x@=aVM(%(Z&bTr~DN(D){F zNlq!DR4(nDUn4c;s?=jb@w!)dT~kA4EHRWsN@x`&Gw@}pSVkO)G_*S*C|UM06DTrj zLXzY@RjguV()OH}C(2fvUYAl$I%6q8lbi8Z%`7UKbl9ep=N74rJX(ptlu6%3p7>$p z>aB8IOr+xVLnP}8Mq93C(#1@UR8_@_A0!r4T^5`n(K4-bytI9#R6+796j$C+eCLcR zQ?ktFqODXTODS+n>f_+;O4T$*(`G3+x$SykiajcG4O*n?W#p*hbd8@$t_L+#nj%Y^ zr-d58!Z-jOnAe)BFcd~#DLuKRDN%7>xknniLw}0C=~jr30W2fdKG=cxM13W8hh}%;X3_Yxkh{ zT%h^D^9uv$;H1fEJkI^$Vho;m25@hwdjB+I{atRrwsX|e2%tAj>8+&!q?ou%AnTu8+b?heWa1^f^&;D}emF?LzkqD}*&~TS zY0eHn@tj0gSl(%00>%ir<$#J0$snNdN!!04zXP<1ZcuwMF`xJu&@ooXkx;;ZF}$4q zWe{6jH^BI=^OJ~gqQa=TCV(5<=BZE+2p&K#znjIU0{pwP{z_igG0sBofOjCED-w2C z%dcTOg&N6$U^rC|oVN}qucbEbBX)q80rv*PYTS2ss&)cjKckTEFXPkWm=85juGlp0 z1-&OgNi3S-G~hj|f!03EfsnLgzx0Tm7U=vHcAT2{A)X{Yg$-1Iad8Cnf$>1+!1ldi591M#~YsmI}$NZ7z1+(-0P3S(M^CyiatnGFn%BxpaQ=6 zgc#hnpH&}Y_ondhCBt*eI8N7#dhX81Q39qmA8kiaTmYOkJ)~XL)vkmAEVYH z3PBiO3>skiV0nW*2wp4YAaSlf_yBZ;{PZ><=o}E~>~%#yMhXr*P?)_2A=uitpcq2t zhA&>=tNc~e{un4)cA$t9sFm#tOJf(REZXH!LL}>h4;A|Y6GK;C$0pr=Ycb~HwZc0@ zM~ISAZGQN;K@9az^;uAZPAbHRcmepTM$Pd9Vr zFONTBj6#Y=r33G~cVCYFm|(7#bzfmdL9@SVFS z0q;ZrrFP0^5*n9mz}4dr5e!<(c|j8f6Pr+0tdoEb5QDA;;!6R1fd^7QDNG30PBLnE z@`+}cTmb&Sq;tdiL*uY!cWjeuD`56K2!ZcSpeuvL)v+HofaPpoe+fQ)*&M>^AXBly zHa;42Q2^{&1wuH|P>hjPQ%#9wtrDQYFJ2St?D28rmqL&@gI|Fka)bUV27>HCl%ZVM z*G3pFC=*=g2(G{ER1F9&#C}cF6kMEoem!VF`dkRi;rjk+^V|-{MI~@k6h@u(lAW`6 z%rJtxdre=a5r8tYh{g#P2;6qP*<`_kN2Nzv0N<4h1n$~TWE*TIaE;nAF}79(|_B~7vm5++g)zy;D^z}-jeQ15kB?%VBw z3<96OyBY$5t%04(f(#?@(W#zgM&4hlT&1zi?3B>VL}#Wlo~=-QO!pjLD0-9#Ad}D$ zIbk`d-zmVlVi3Xrq+W(x2w3$z8P$qh?cH*G-+#!BZsmcs;YsDvdvf9YM`6v8yD|st z&vTfDAz-L5t~Ep6UH+Wz6kYYXL^ps@s_-`2PC7`d@3~lN6W!oFMmL+D!EB-?ffm#z z(KZ=p)(zd_NyI47R})6I2T%~^J>Dmzp0H&xJ-@%w4n2My>*#3ANS}rFu}$=ZtCIur z1^0OOTP-CnqX0cMpxX@SrDLKOu}g%2pJ@kR)I2lLvNw8#W%qS7A=@dy9%hHk8Hyqh z-4_H@4o+M}v2TSW*xoBj`IYE_EjOjZ3Z+ld28W%ElRcF_tE1UK8Ik3ft-a!|caO9h z%`MqJqDMKF#g;!fCfVkJD{2|l43yWWd->%euWj~~gIx|(>s|W#r<*081ZY!E9rU*w zdUxA%qxTgj*q4acqGCk^m4HJawA&R79$QjW3L~*ec9VI6+|TVAbq*!>7b+98LG`m) zp3lk^cgn{DHS8OM&v@Udn8~=I3iW#^S~Q?+Z(hm=-6_k^IbLk@R)89fFbN{U(vWuP zk+#q#4>SgA{_q|VLt}yBA75kHwg&ouI+9HSAbaXw%IuegrivxFo@Jk0!z#_t>KwX6 zA40aaYgRckE!cn2w)85Pyw2>-Pe`xugZ8AQ;HT2TocA&`2O5Z90hM0ONXWB@CbD00gbrTW9!AF|e{7=uIP)>E$g?cpR?)KV z59{#_2@F=o#2Z60+}QAJFl18;L8F7bRVpqpRM=AMz(%02gW``6OfvFH13Z;eTy7ru zjq5&DN}3H*j4~A$7-wJDV1neJ+ZDRK7)%WbMTC0KFVFa182I|b#3P1`R2QC4uk{7e z@rB8iHJcB?%vgveZITg5XCa$I!0N~kg`W|(m@1FtSp-ad%`7a- z&lP_E$ZYlcJ%w3JF*gkAT@K8x!?UqUZchWyb#Uo~`3Dm#LqjLwlh5y}_QhrE80p{k za+ea622+FuY2f`@%W6>N6cdplv&xV_CdSb3Od0KJ{Ti@h-%qPzPwWcU>I$#roBKCS9k zd*hIbzq0NHK0iS1nlgr&r91`&6xh_^lP&i2=4E&`G(;`uM6g*&UzsSQ5}aH*VkWdJ zD6EfP&#l5f@%y=A*GsnZy$h7h+*6a%-*<~8rb!STVLGmYsHVdITjZn0Uy2~|hy&<_ z@aF^LI}>p`m(;p1c8dd9o0bFFH7X{=)4b7eh=U|RP|VYMV^z0NlQUNm^+e_z}1N)Or9cnX2^rMJ*D!h<94lTsS0op>rlE`!f9%= z$GZi0NYfVgw~JWW?FZ%pNzq>f1A^c|8}Ap|Bgk8zdfM+uT_?MvUQVjPJJGYO0f3%Mi zKCDmRm9UWXV@bO!NqB;5h>zscjdjf!q0W^7Qik$=9Pf!U>=D2<*{?LnWg_-nse(w_ z7(Ekky$Ly%rJw?bBU}>TyfKBV+`}qdDeHVbno@|#liL#PdJvh5g83PNRs;t(e{JK} zCWy}bEpEgy<5zma^a2PnYY(h${ahQMX9UwRxF|sx_#PuT2T~qAQ}k@lLjD0ERo9hB zS-HSp0?!yfIt3G*5WN2aPVYP`y+5^}=f3Q3A-h1iXgvoZBY_(^WW6jPIMgl{POgR-YdJhSu%S^-N#M8!{rzU%k_Rzu@?xKkjZRT|Ao-D)ozTrV zz$S|$_KpkjUQAZZ2b7+KG6FM3-hGVu6Y~+NINu3kXTFg_9K*5PRfc{sRmNvwIAObz zLIp|xmVXY04GO20LlteY!iOJtTKvv_Aq=9}gsI(lI|` z;2@nXGs-`~Px~W0eiBAH53S|l0{6Vk8&O(FdfCLMyco1~G=3==?2o`-fmW_cvJWD} zOY+l0GJo|WOCw*9dGP}qc)S*55yHn+o_r9wePpvES)R-37E!Xiy@v(KxyA%;QZu-WpLhFg#yPNQj@EuB$A0ni>9V47%cvZ>l!77c zSvH1{Aq|v)sT^2WDg0bm%s!bhYLe9((%_TD;wzkfVQR`oh7><#rW1Nzj+BKyfEFbo zIw(1Aj8ZAS(G7uBTB|>x+gBecv<0PIqUt%aSpF`G%k~9EJ1ChP*ufgc29bN9Oe!Ne zzd<(10kqK|x!lkTH7kH2gV6_Z9$WXe zGkn_01Tixd4^&X|fCr=2o=ASBHtAe^_B7q|6k<}7+IIZyOq{)f{$x-VvE@o#T0`M? zAY|*`qiv&!mT^XBsmb3UyhQFty5Y;V@=i2}Q9t`y)L$l5d05Z5-vuH`^x^adeT>KH z8@lxZ&$A&}o8D&l04Y{M#+^4uGWBI|(R*MY2^~pV-DqI&1BlO;owbc}_7WVCGkVfT zdMia}6IZw#ns4`LY=f{DeL`~QpAYF#v>mcZb$u#-H$YzlC$bFvYeXbz?cd_@?lW#8qt9C;_ED;LeJsl=Qnis%m0HWSq*q91i! zno2f)b9t(Z?6G^()*FAE>E~~X)9-e!)<^fI^N^%( zu6mPw+&6rEcbtFEbd}Y+Sz#XwPp&Q>2Kr-AD$>5<(Gsr*z4r$l;Hzr+fAj8S?;PaGpnaZG_aF<^@ z$a9{oNAyh?gKvqp5^xwS@+(~#EDbjLAu+&N$Y_|WJ93&ELs)!gglIZ~dlK%HA?tK1 z`8ibvnPE4s-`_a~@cfgE%zU7B;;~NOMC$m?QRPZ=2z+_5zdA3Aon?A?vHGYi~60uz78f~ z_kxT&0n(dQi!gh$1@=tPdv!apY^gixwtAh`Zd(8^Cn&xnc1dm_AIj+S0aV1}zy|pRni|I}6m? zV(g73XXj74-*6wY^QLY@>RCH8ITvtksw*_T36JU*zAJ-Wf==QDJ?M=-C?BQsZbFn_ zW}*FX&Kt9wPyrWY{TMJGgagNC_T5V%`59N{V#&j#G(%q>s^ zX3V@zawLT7*}LAL5En7BE3GbqU(}td`;~;4b*+lnwf(YwVc!eZhLOVx8rPIbo2}e# znLOae;CC+s3x_X(*9Q9mdvMXJ*5y-+i_M}t)*%3I7F7(rKdw;PBxMQGsZGD(1>KOD z*m#3apPpFigDCnoEWHP%=^Ku#uuNK7baiV(mTZ<1n}HK7%pXw06|pycwz$9_S5C%} z6)FK=PTSu*!mS+SB@yVvE=H=$6L>p@oW5_cvoh}XnUw?MWfG~kahL2a_rWS=w`2nt zTs^rDJ7aA~Sopbt;~E1tR42Ev82gOijHDcf#>QsWoElQO=Tu)0u7FAEP$f>Mf!FOAA-CRB)&(-0SPBN z_bDa4EHS;6npbil`MH2?h*FQ9xJ*V~sFje|Ql<`1PQ>QW_P~dDl|g38atdL|YiH;WWGv>&JwL|bllS@R`;&P&l`yY-8n&^1CIpTKdc;<85gVr@L9KU;N zRXySO(9@84`zu7DHeeUYg^^ocZoXk}}-XSNyh!rKLw#QI{YNY@TwMLoayU96y*k#UTXUYgl zPm#vM={&-~NMhwjG%9*d2bGwof{kL@rPJK=!gFK*t?;EFQr#>3ojH3>%@(n$ANsA= zUw$I?THeX1qXu7=`rXV^wiQ;tcq-r=vV9ck06BlDdV%LOtb@-cf`DWn&Z9SyGROt! zPp%Exq}3NNs9Yck4s20(jJdE7-nfuG_-RC8w=|fLBe=Lu8RcSDdwF#60#1L=xfI>N z4{Y9Zk!TrLyk{QdX~)&E88ob4)V@EDf@|(5|JXc#(eTNtO;(!4o96N~gG+O+$Y`kZ z_@nZxEqxI`SDd9(ZMphTWXMatVQ9!pn%o0*{&9x_{v|+(aL9Ty@dbDC;sgn#HHFis zH*Xj2Qb@Dli+Rv+*Nkr9$>ognRf`MZy!|P}=W{;Lo8Mpg)xh3Hov`CUR^4LMaH76= ze)A4`meKtwWR8fBrFyR#hy2N?QCEU=nP4~9LOz#e0bwhHwFuG2?;h7hwD89^n;g&(}r1Mosi(g?=Jb^O^^<0kYYE|2^(^^eQwDiG!-;|L8j`G)R5>fsd=#Q8Esc-g$^TbkmM-GDCXvvc8SaXXN-Ry}ylg!QPT z)8o+z{6|I~AMJ7Sfi0|U>=Dk}*S1ZYfO;(L1q+v;JYqj2jnTOXSE$!J1PgkBEwO+KGvPe-uf0@|SZaoCChXa(_Y* zRLUvSuEB7hNwjRj{Mdg|uBU4qpf?N~TY6hT*`45ZQT=LdpxoD2$ zuW{U*i;iH45*E9Fq5)S|bV>)}^mDlVwY-Ra#MRQgc6j~eLJ_P=eKYnfxLU*}+FjM?)2!CUU z^)JFkp1s5Y{w%-%j%@a60RiM84$q}YbVq#fB{%@@+_?ltX5eNx!=Onh*0fj|hen|{A`#r3FCi#PLV+b>*L-|^@ut{^KoUt|BQYHS#HmZ> zLu~#{;xI(3BuQPIhLXyPp&KN9XYAnU#?d&NVfD_zNuo(IEOM{^B%}J<&K3Uw!;22y z6feoLdN15QbBKF&OMT}so{*=D&?)d)vYBA79L*_%u05mCfun37l zF@IjJBAw+Ofb`f8;8Nw>i!$0ir)0?B4ISPtH%C$kgPff=fo13$lMI)?YzQAcZ`-DG zjxaE`kUV5Wc}Wi$Q($ngWo8=_onj1inMCDu|N0e~_fwO>Q|2NC$>Ug*j!F?l$dI$1 z-Iy#K)l`!&HD!?$toFFG+8y98*~FpE{GO&D9b^j==|xt2LlDF;lXd34WQ%hsfPDi) zO}Z~iRs4{xoY13AxKCiC2ihOdu1jV2lyyRCHuKq(4 zt&kI;$8vI)a4*#pL($8h?=EAfXypw1IktEHDhU01gqx6y$!Rk2!E&|H;s=2dL$r!H z>iZVaNN$kS{)&4P+*5&5;gQ~u(j3x(@d-V?%i*7x?gWSLHi*IzB{0c&+HEPe=gkz!%k?x>$B_HQ@E zwgRMt&`OkC^mV=f@c~Mcxuc>z@m)lia%FxL<#Zc3ACU1=3m+6qp;TzJe8-nNK)0tNvFIyp&}BKy*)_%78cf&ck4BRtrVpS_6H9Nu51|!3&u8C z`2(NdF)W8_y^yZ#XPo2Wy2?WEFQLlyu;~6MM<43S&pDgT5SSvs$)7)oTh88zuN*tN z6s#Q(_gxMso|Y3Snu-A^e%BQg#Ecn!uHrfcjUrUMu04^R4^1_dYzvm}DorF`B19JU z_5UX(r_-t|+XAs*mACXOZs?7+mh0m8i;@RQ>qZsd#9=rM^a?Q2{B+ek;q$bS5Zqg- z>UavDt*2{I)x*7Ksmmo={ssh|=QKjus0MOOL>wIe(G%h~;f#MC=cb138LJ7I)GS^; zZvw?jABCF1I};O(c!AaI=u|E`7hV#D0IQuU8*jR*^!5&S-1h>8HI@7Ha9miUfFGR}k}m?BTzmQ|3Z zhotHwb^A2@gIOmfc0R&#!pKoi4jy`eKvAiu+EF1(8sLdfbM0I-*qb4wk1#4{`B0wt ztHG(b6G9q|*xKw*nnuC1(l`LB@s84>q#uo^P<+v}Qny#=dH%*t6UbN^YehRlCT$TB z!-FxTNg&+A&=!a^c&n(N)9Dto6=-kb?}cfSsL|e8O2+=G zKB3x=-R6M_kU;Ab)a2Xc(cP6(?Ra79Cf;ZeDw;6iIs{3K7b-A&y?LtiRcEZ%GCIV! zMKU+D&V017pL=LL!KqI4VK30GlMa?)j?CY{Yuw_4Vj!T<>EjA4u252nTYaT)-wTjn zof&(sjA0#2OREDqViLjdj956#ob@;0$!F>m}U)56vN5j_YS0Q5l(@ z#2no?ApT)X*VD^CRCT4g-PiY<4hUIh73hJ0eu=yYraRKJucVfnqEbD7lLb7-h~={0 z>Fwhg;3rOtMsGhg{;D(62b-Jy#>6GVmwU43xe4^vpWOntAf4C-nW(!r_@qfi^w%w> z;i$hp)^i*gkjE$Edc%ADE%<6%G~O!%8z_X&!Lm+oc(NJQh<8I`VCpI@JFOc?0b0nH zpDhByz{4lj4SZIRgrIw!*VHD|i48*K^Nx^Z#&m2A@}$J32;Wt$xQ!pzpv2k;XzNSi zJ>8%W0b@GPAA=e%D)zTu8_X%lH&7{U`@%57LeRubI%5c;+655Z z!pt}HP`n2dZJ`~|PWr1zjffsQrtv=B9!a6&I05}5<@YSkPp*tn$Z3hPe) zF}XGw&Hr~Cr3MR2WA&>Uz!a|XB_8^c9ww&n)`Tg)Fn}z6CbKC*l9vn#fOAu{Z)lTd zgQC_+L4+xKGoRn0Xv<6^-#DBf=WdGoNJPp_37(@F3PdWtM7-Ihan%m<7M2FrM(YD% zmyxM*s9|=LgV>I;b>{iqFna+J9vxtgmoJ||bJs!(KyHrP7Piidn6pHYjao)l*z0%M zN>6Cs3v0s z+Xuk9(!^hWAOE@qKwy3#v#`#6{^6d*^gVtFlAz2m7x7%H#k$O_AD|+7OB^2z@g4eq zpjODU&(A8iB*Jpe+%X~Dc6N|ymSRpB+wJasTg=LjIx^giWjKu66^lm@;wnZ$LDH|B?$hwkxfkdcT{_#TXrLvCJ zayn^5;?*1NH!FI)SWSVWTZMRPAJqgoSSy-Yh5smCWWH|!7gp*0wQt+Z2y64L<5vxo z6SKQj*M7vHn~aS^Nn%;e&}4f^t!CM!M|FEhzjV?(>jn!FCH88=E2_;8^MF34*3|0> zz^{i9Skor>R9a8&&0rAyujQ_2h&ovx@*MtbMw0jK-CH|(D8sbZ)=&H@>zAUF)PJ0f zUnmcTnmFG}<&dUYr~4uZZZ^o%6vSeK_U&k&L106wARE?~;nK{$LW_rFKqTbZNL$dA zRnIGE`WGY|HYU@YVMMTvbp_}S2W*u$-{kO(C(m#x6f+V zcD9DZ&Xu4~(!8;2!O=2S50+xMg;pR7+BJ3)NQ33Jc4QBC1KT5aLSX;F%PF4nv-5zW ztS2v{J;r>rpF=C*q7#u$-#w!{wBLVR1&`l-U%RV8q=zZqF5MS<;~MzHxa7ILX;eQ3 z`?smBc>Tw8$v%l9eO*L%Vz1`MYG>S%H}V|BWCq^8#2HM!Ra-ja*FpC+s~q$-W1e9~ z32+X!DGr9=Q?z4NslNW^$?*VtLdUb-;F{s_F<%E|Dg+x{ozZCf`W^K=%p|AW-z4ar zf)QAPmK3Ik?k#X`a0w0jV0M|_)X(h?T&IlmEo2HAoI8q)yYoLVc(12)Bb)0b==O)5 ziJaDm$8t6wh&i(q>w5~A=>$S@g%Q|E!u>%7(bSp$S~`~7vc7a?zF zzhu_=@jm$eh-!9c*`afAe(%}Yj{)-dDQKldm9x0O^Ro0Q6uQnXn0m?vDCPzkBpu~b z=&tI*h@mfXk=PfPT-OpyJPhW&J}xODL%;^J)FqdvB&QFyq0 zFPhh%n`{KrLXv6V$ZiUWw!0}kx{=b(6@`Msx^VLY%dix(Te8SB*hjm|lGRA8W6~WF zxBRg305ld4Rt@g0AA=*wuiz{x@&1E{!rjd6s`jy6wiNR78d80`HSwjzzS|4B|51xm zy!f{oB*cjg)< z!pc(uR%X@y#&VRqf3dbu30y$Pes5eYYy1FkcGti=urX(*#vm#Fv()zTfDIS^`p(k6 z{TZeQ{^m5pK5}X3T@w^|_@XM7V!KZ&ZW#dhScTzsj~MxyJ^Y8bB^!(>TK8Oa)YIs;XWhg@;Q! zXiaxRe6XiF6HMfTE-Yz0-6U-2VdnkN@wO9|2Lb_4uf5x`{`ivur2OpV_4HceU zj_n`ct(vgVG6Nc4dEw@zJyL!@){N_k#UHrKUN)f4$i2hW9>`uQ{Mmvk_KK(3WYhxS z>GD!sRY0e>J-=Rgk;cf79BKd6of7Gr^37kz8$6kLm*q~2UqbdMPEz?4P}>C!M?Z< zpV#(Zy#%PfDvYn#Vk?903*1r>YlXjWSJ(*XsK7%?fj0QsUk5W(2aK{;hme$Kl?nqx z1SDWjek_C&!n`=(cN0JY{Vv%M4CLnAAD0x5SMLa*IgbN!5vcB3THe2>1{|QOvJuVBn2H z54#T7LIZw`#$%I|^J`Qeq}RIKV;gpgdw5keU6e<7^qTj*rP zU#yD6d4&kyqH_z8>UU27LW!%k4#G=q$^5&>u$2!_%#}6REx54D*wFGq>GwXmto8I zzV>)@8~zr8NhBb|-{ljv4f3&~K=-$pC;Kk*x<{R*e!*J-Ci{z><((THW_C14Y!&PP z_S(1R*NcYd=hpCOx@Iv3e;ZqQ4`SX=;Sr@u7R^#JfvwTsmZaEB2pV}i{^@{G(gRmIjYc{aIy}KFYYSg$m1|-(``!5p41BPLX@hiJN zH%i!iqx>^J^f89{-G;#GVg#0Bc*W@TL2D3`t9SdDL95l`m>s46>zlt=aGJI_Qe&aL z(=m&stfQ}3-ZVb^-Q8fkbe6>O9y*N2sv8s+EDI%2qd~rM{n$c|V)LGe96MLqGUL(< zf@qARCZcirZ0_`6z39q|$m4=Jh{TtIG#8$}fDufe>pqJ)Sg{Xh^WG?wXatYx=zAD%2_ck(RDgp*cF`CpOoMDQR; zBriEJV5LumU7Y7YoPZECCCbQgqW*-=&>j8}O>8l&601)|Vu3pQ1V1G7-t51MIn!60 zc)a~12u!CS{3*sq;+1bRiQ#WV@vDQpBF^)ZFrxcol3VajXDhh^s~S5(dlKq@@=FsD z7~0?&_~GcFpd?8v6!vB=UpRVy>ZBv{IgR@wz&B`hHR&(C@1T(kP&YOQGyc>0`6bi6(l49k4vI z7o$d}(x559X_(S3nD&L~r2D~?cbRbt|8HA+S2;_^a<1}f(0QifZLOFsOsmXUa5Z1V zn99xZ=NZeD4Y34-Uc!RE4+Vp*rcjL$hG5{U2G9*@WQvK ze5ZQ6PUxV#{=U;wSBx~okFuabOGD8{S{(kg!%-Ec7F1I!g!^Urk(In%UEb!YW-dOz<0xwb-}x$vyyW_w*S= zq#6zLnLfYO)H&(Jy7dZxl>PuKKs3}KJv4%uVGnPf;a35MjCiY06h~eJ$rxnTLjawU z*vgpjUtCojI}jNMS)lw0nZGRlmvQ_1SKh6RuNeE#NV6-pn%9U;gGy|Fvx}D_H1r77J{>4+7{0uE;a$SzPWGGCxv%`B4D+ z;~Jz&&ysi|u>yy8)>4CHeWnuV@q0GDKJ4y|*y>IB3NEL~kdw_we47_QELMKm0tD>^ zNiZ&3jK0~YA%3>4lbw@{`pD*9cAi#t(U+G466`wS=t}-M668ZoB+K>FmKvOs_`%rf z;o3ZCPWd6c`kH9ed4lujbtV8|{oXqP&qHWe;K^PNJW|E`a21rrtR`u!W`+*?Mw`w%4`PKST>pv+U7ljKnXKUFVd z!z86Nq;_6dwue#PTn`Q=;MH~Bt5`l;Qs%LmPpKj-g5rzw4*=HM}Gi7Mx7oO;^`ii)QNx{t5TF0k@0g?_iKhX>`EV_Xn9 zXC7AT#kz^}rBesb1u;jNK9#9 z)s;LAEI#A;v5vebj8GEAaz(bnWz@uW5&bc40pqVoG0*ReOT4K9%W%^hr)deH}5aYFOMe!DvNLR{iE;KgF55qx?!SG zR@TT+oc9x@!}s^c;RnI;qT%v}?^D?UcpstjrLtOf7vM%{hg!MmmpZfxta0{3$Y6#Q zuH}Bt3)*kl?Wjn38rZwo{6)HuunG?Od)mzk?ibe~g#$9OAS>+%SHzC?zlt<3h2rj- z^!8Yu75#$htXL@Q=_34$e?@^yWSIUGxMOAJGN`QM6IaeKKknke4-O;?+__hU_Jb_a zDb|&WszjIJlqY(eRsF%p(W~Y)V-crmwQ%KOm!W^afhg6D4Yi2^UFWy+zFj+v^|fjZ z_W$<|6tOR@+Ba&;s*__(tG`}dJL4nDsd}WW^HzLvKE~hblvys-63dgBy8scl1nN#ILtmU1PGMeO4+|N^lvp?*{g0r9t42*i?>wroOC?}`u4&hm z9QQTp*Myxc5;!6Z+_clSo_}>fr zT-m*g9%@*G8tJ3(saz1K9X?XnhI;h3qTuNZEzmrsL$4mgw`DIL#o*0Z_+DQD3SIhL z_0V0PpIu95v<2|fgg?9t-Ng26YlGJvctXH1#==j`;z~jg7Od(qS(>mB7XX>-9ZuU< zBpkT)BZ7pK%!f#0esE(3!R0CUgrzgMr7N;okbo&5rEtWI=s^j z8d!^`uo+-F-m9S=cm!vetjWb zKDMyZAE@#uN>&&SH0o?&&$@Eu20#!82agrVIu>oc$^8A(Y77e4CT1)6wU5Cce}xRY zoGZD@VGP0L1!9C9rk`ZzMuK4T7b!6YV5c#I3gHh1NPf1rXUzR;3jr?o8;toY<`u9W zh#267_`Saoyb&+XVgU`5wA476aF~YKi}kVa1*V+F7jCCU7R3g65+ychd!1qT%x&<9 zaz%>`cK!M)NJP=8EPjEb#f2aYHZ)}#83S{5wnrfar-VBYY;eyWFe4c!Ww=u?S+ozI$SRH+JGv3wB z`N*2C;q>JX1igEjKuAUYazx2QWkqPiMgrepC6v^6S)n)X6NFbP0Ms_tCIsFR^hZ5J zDfcnq^Ww%QmBq)W9l6Ff+|Lpul3C(9Tvt>FESO;qju=7`vKH+=`;19q* zmwSl_=;aM-g_*41tm1?h{7PSxeJ9_HjV`KFhh4%tZQ<=!81!9JE#)#k)y<_ z-<;fdV$YAPq2e06LzmoE<_xm8mP>d<0@5y=vKo#d@ zvrKY*r$}Kszdbe1f28CEQwa7_w0r^|K2WcwL=i%59C6p!?1Ud&rerdd{h9|YbV$}9 zMzVIer0gc8?`lqOBH)x2V=Xn{MnKA(zrU0AVD6dm!k|w1e4#I(QJtx|<7NbYQ=wVn zxA3)BQt=yEqGj?N7<}YbtWRdDd{*^=Qtep6OV0={e5^c5WTcp?WBO*Oi&F0Z@b*z7 zWL!zzq$C-fG#D{{<8RH<@6!pd=wzk1OlaTRI1Q^6_2du?Ycm<LUpJ(wF=Q4q z(~jZa<|9ktELrce+=d;dvmE58n@xW>(-nWIoegBxHmarTI?foq*@qy;{?hGFj}T7x zVShSToBQJQEvJD8Qbl@Z5RFB%X&GcV*2p#cZco_lm^mY})(l1*Gf}!)h8}~xs(R%& ze~B@*sWYaU$OsWJaRpR98B^E;B^(N{Z|Mq^JHt`Nf)A^1nTT57uhj&(ZF)QTu8$CA zB86;L`KWcz+bAB@H)X4%W}@_l+N^kX*#SwZA#66sSDPZOOR_6VLkDKr)y1sSE(R1R zLDA{}D^r>MePitdNT|L_pZ&c=_6$-npsF7CnI`fJ+BkCPvo|(1-aJt0ssX2$kx2VZ z)i077Z`G^3Uk7IlsBq50i8zo0>ykwugyn_PNvimgWiE-`>X)JSGpkncbpU&vp2Oc` zEi94Mf0)l^eOv}YisqsF*?BT9y7QTX3(M8_Lzjkqu2Wnze5huWBZBXOUC^o}7cBG~ z1$Pf7IZ0YQMQ3k)9->h0a#Ee8$?2mttw|vb2ub`6Mv${~?Ur#|Us^7AA}1T>_glI8 zxi+s{J3j%U7TGa{!g_2pH^?s`^?u=k;!aVrGH}~EH|{#W0zd!nQ!qUzL4BsT`76c= z#CJBmX#P6Zn;R5$K|eP1pSkZ4eBo?lyx`zh=HC@BUg0mlA&>8ODGgu#I<5)@_d9>p zdEC`VEX&8(AU1J3vjKuL&t3*Mc)H{bz~1!dktmqVZQS=PZ)V}lgPIS6+P4mfQ>kV( z%g4OJppBVSnD8g!q9pmUHYPu`)aeQU{zC6VTtVjizHXv zK(sbF7_z29{}TQ)KgAcW-x+_?$w0))Ut9Bp2d;NtSnjl)plq>Db88DLaG z3kOZOP_~OF&=v7fnUIKbKeXhX?gx z(V^aOd2KAawJ-eR9PhvK1~Z%I9T@XL?;W(|6YNz=IX8#WB>o(d z_JSusFQtY46r;2$)N4stD*Ju>fRrE39Cbz);+Njv>-9$1A8GYbCxKCSe7n;7V7v4e z9C{16oES+^+o5Q7Qm~qq}zd%;O<{TQy4v7+00>|c3A*9j2D&?9y;p{0F-N{X457}A@s{eYjRXS}^ zQR(j|ZmH62$3uII)z>JZkU*W5jNjs4mh%58B=Ahl6>^tBD)yZ(fTeeCOo zox1jfM?{qEiOS&UPTnmQb>srRig>BxKj`eVE(>heb)?L*c%+OvL0T`*2Lj;s0q&2O zW4+2l*I#DhNg>ZC*oe|M-3CDK_bO%b3&fom&qW;sYp=Y4&bc=TjK2mE@CmQs{n6kG zZu`RAAEPU88&Zw{I`970@cK`AHw-e3JvH=XG6*sJA#k=x!~b*2eM_Qi73qz8b<(Bb zW_r1^V9jM}B&czQPosco62#@gR)u%<8<4{B(J0y4HdOi5R=T9zEkhVL0dADu{`T#A zxyk$k&-U+jQ5yC8jkn+DKHd`yefr@;_)}?f*93Fhr;INKZ!{cpMjE|_;$<}EIRb%H z>jvA6|20Y0SX}ecsCdN8ZCoQrtQ^RE_sD;o%^Wm7tLLXN)WScr_0ss?%L5ACC>sU~lrm1W_T{b<=#D<`jr2GD9+ z27;Xd7!Z>)pe38XT`TT3I zmSyDlY!MM?v0L;LihvL>v&H42_!@1c4+A-%ybH{GC0dd?#zDri@N<}A)XLtSJ3fI{ zAmekwG|b)o>cJ>Z{q|3QqVxJys?uu&1*3D;UPwE&m;OTC)`EQs;WJ2bW7sO*x|-?8 z!kHEH(Nib<+{H|NT-rt6>G z9x_wA?NOC$rHICmnX7~5YY&5X$;~#Cb`HDp(;)-3`v0gZ)_X8d`2_>FLtO^CScQJo z;VzvZ3?tGpBYoDL@Gk@CL<%y-9Uz>au)CmB2;tfzNwycks4ut@FnGZ38a#7%wOcT? zAu(~?-9XHqfHXaBuHatiBH|s+yBi`Ea`U6RyWge_sPmV04`)`KR_eqTmO1?1gc$kC zJM)HNT2w#o85ix9br#TKhyn?J`Hf^^jvUZ2y>migtP?h^&L5!o`lR#&1i3$vBI=@8 zdtO)EWplKm*J_z2UAo+YDHlhcaB~N2{DH36gwi8v-Tk_U;QxNgQe!XY9XhhfKc9$` z9-c@@ehZv-e7+XOU{`na*}lkar=n|wc-^6w@#LL9zB=t<)(y{B;iI5i8p6PHFgBCz zk^Xmf>tmNsx~`SoW(tIA49rl&{Lyq<1-_lMX0j`{cMs7y_UMD6t|xCc7(zJcjs`Az z(riBjD1JYdRFV3&YkO%^<3OcyC9dS=Ninnh2G_7c2+;0^>**J zrduKncnyL4)(ie2mOl1>)YMVp*Ikvuov#iS3|&Lo|9N)j$!<;oxU449-CG)?cw;0&+xtUh3sk% zq{CC3@7ta`lsjJGNJ1Xsbub_mVB7wdlS*0XCG_{QSK!x&bZR%W$PEZG*t|Qmj{h1G z%D8Ls!C6!7J$T|zlFu8M=MWf^#eU2DR|apvEKwx-5F8hpAh1RhoTE^mVjOHDulkrr z6LHmzZvaW>J)b`Thk~`r(ctG~o!$$IEA@w_ef3C$K%tk#Wq7>|Ekz4KzpQA7K}T1v ztYzHD4Aa7zMGo4#7e`kP^N66-sY}pJr*t}946E{#xot3>JisO-Bp7zcGlf^C z=i}yrDO(?IXbfj<=4$sYEee@E5V2Efjsb))=okR;2!7%G13$BQrF%~?@@{F8Jo%$49>_3sNN;otaHnI%U%MiV0kzHR#W4*v>qNuI~y z-X&R$mJslPf4Hcgqn#J(>*S%bt}NwVSoGt!zBt<$^2B@?<7=OoKPGen;>-*BS8j?v zwv1A_1U!cEf>bV5Z*l3Jj9vbu{u@L4Jz$mHmA%sfsW|tZnzkGFnBLI17&#H|MEe#! z{_}vAoa0U~$T6)Z0JS7$5xRW(2_krzB%R*3nCG7fCA!=<{PWhgn9y20H9TR2$8fo_ ziDZ?e`i4~6ium>mq2C{Pq9jfy{+wRvB>pt+6~9+RcGkTyxJ;^H7l(S}{H^P=C>$D3 zS~W|4ZHWz;N`NFAI6Z83%WDTaRVO6Ji^m*+;%qneQi-xW%>&p|aEblFD191@!SuD9-aYSAjvD&YbX*Wf zuEFC1AJb5bpu=E~ms5hSI!#}@haI!NbvjL{4wagN=l*&4c($KvJ_9U@FYv?+bs$Y< zh9f6$`iyL4-4UKKz22f5G5GW5$|x7+iJ0-5#UIU`DVYf_N#o6geK%=aPcHPwPX0)0 z80|jIG-J#*b7utwc;Sn9XJ6TjaOOcuZ=1CrS1>F0D!H*)E3X4Ja_fAnq($D7%(@)* z?G&{Z==?exuvie@u%_O`$3@vQo}&4UP{Zxc8u4s;&eKIhv6I<8GJ--A*@mF#5!<_6UGN$A`si_ISdnr|}q zg#oyQuQ0&j_*zjpj~GrRZO)K7Lpx9SGY-}W&udOr=WFXNJ)K!Tkkvo`%AsNkT%<{v z^V*L|G4J2UjG;g$TmU8vVqSBj2Ihj%O!=i+NGLT%j;;UxQ>eR8W`_4gDmFF1(RkqZ zA11QetDU#S#UgEh8F9C;9~Kw=P;i~zZu_CVVQLo(7cF17b5QV2Ukfi0Q!WsC@aO)- z0}KnFX;LQ3h4SyBnlc^Z`*K@s#KP;Vgv(P}ZL?bOx;);o^MNlHXUHPUlSkR2GpH;J z_Q$4!p&>qte(uOb@5L*H{j-Qe@z09)CH$KV4oE@?`Uo`cVu?dURAwxR7nDo!=Eg1M zs!jhmY4Hq}B6&ItAUCjw^?k%cu-9OzKHWG&K(58oI-IU8du?aL-mouqi!X=1+YSfP zdX{sc%6=T_N_C23eNu_@@SIU{F`c@XYpzT+n&K|QhvtgR-Pt3zGPJUX4IYK4)dHx@ z#nvihepXiD*GT=yw7iPYGm{5cBn`8pJ!V~6RwMo8g)xXmz_DX!qK0kBI9RU$*(F); zR}5t<0qaC6aM$J&a4BQ(Tu0AeC)I!U;rsYR@jsBXCVEvP3@U74lWZyO4 zwO+ZC^SjVhRo3=>)}LB0%Yt1FrW3S|s2J8In{>+MLIYSg+QRwvgGJ^Br>MgLz%5>cDA1oi7|CG<(;0I(I_AsOu z%siX)YOvt}Qkid6k?!D~~ zM5;ZS6c~*jSN!kqLz%_!_J=63fzI>oPh!l#+rzy7Bso*Uw?E^1z9{+29gU>GB#-xY z2P>B;eLC6mLv*5Wg70|Kq_5<05MJBO4x7>5zA=}PaQ(~w{{_AMI&{2yIyrqLw)C;X zy5L*dkCjADrMW7KU%x?G;&{t-IfK+stXVTRDaw|(Ww6`YEB_gb{sl2P(dupU7iH-8 z0^3E(7|h_dgb9bsuY}nQQ?SI(T#}oPxtL@Bwkn0NvG(yUhUX*fja0Gv1_(@e35<7`9b;b-UChhUJUtsm#Y-m8odwmM{F`{_PM8>K>E7 zR34*3{o{^|J+-9(5gkJIlfQLELU9;bhf)D!FLqXIk4W1~WmM50JcKLq*parKy zA`z%E1o70SKh{6#UO9Q@90**HWy~L6&0A&P61@KX$;A0&Av5+Jh3S`x?|vS|;arZH z9$@D@FIg#h&6c@TVAQ}5YO#O7uf~)EGu}Yt)pHEBGkPuJc{~ofo$~XHB?8_uL;O9& z_jz#TWQ-zua5olGoWvu%+*n7}Lv#dOF^w5*jz7#ZaFGSUJi_a>2>9dagF~^4cC5>G z=U3^<-LpQPZk;F}Vta@5RQgyz!oN@YVax=bW|s7NJc@?#81N!7nz zW@FQ?t|wd_p?FLT#^+AOO0Zvj~;U+Qpiu1GV^;`Bu#15c6L3UVYKwoGH z4jaF<8J^h(vkwOWxI^71Nn64*uP-P?v)3YknIogAs4sVh&N4%7g8c1wZto3_=kW#z zqT&U)DxM;GR^rN|e@p-Xt7Z7VpY{L$`#}62h`$5zZy^2)#Q)g-|E~wqIOXgASO3pr z008|J09F4s)IL{{F6{RFa=MlY7PY}YJCfs?j&@VDKHBUt>(rx+s?e>o1kNwHIKj(+ zL++*sU~0Ap6X0+BhB)^6P~m+lN!H=(*nptDv$q0=18W9(RoUrODB2VqQ9kobV)HnK z4+|Kjs2TcyT0|D1claWAi)*#op;}Qz>V%k(fvbQR;oc!GyVFbJ0AL(otpV!jR&Ng) zO3Fr7(StmTUs7O?pt;H^O^0xS9`kt^79xN(>YxUmSMou(-RVgHL}5Y2vO|a|){u&X zq@%S(%+@@aE?^X@Z3X{=?5jfVjGc#b!v%0#_0Znw>${WPwV~*ReIbMV;mTZ7AXy85ok|Kw2wHkS|{&rgh)k=sZLu zI7n^Hp*o2}py@g8?U(_~lR@^g*#&oXDX|_IXCJTDie* zW~hA7v9VoN9)YKUefsI|Ls&`>)1M`@#SLh%ktlfA3;hwim%D^kla%|O+eSJ!sQfqL zelx(>43|Siuo2CxP=k`pzqenekwY)baEQgzLada%*JXZ7$Lx{UbgqkLuVm%tnHSrX z1VyXUCYQ|LgrxpeN=b~wsQ|y0VvCcCzNfH(3&XmIZdle()UtQsWD@pa`=Wepmwiqg z{O2O*$oY|o{a#-O%IR<~bnV0VIMHj%Vd{{%GaxF*@6M$uu75#l0{ z%m#sAEArDjqFO+@B@o+=c2o5p%B<8YEeUW2Y^L?VEO(07PR^2~Cr!Z|LU)yk{(XxV zsBzl#K!pph8KyH^2~QNO<)W5EW}uLjD6Y9VmVdF*7ucu0!@lCkyh;;Suh@pawgqbBVj=f1d!#VQ48k zjAhF{>_|;9;b(r8=;8);KUq@HyX#QHW7srCIr6&vw8;)ax#v1BT0>QF*`8umrqe)- zilD5^e0lL@eenXQ55MdmZq}GQJK>Q*Z}?P)D}4E+CucULnn zUV=N+yf{Es`=043f10!QTl&N$#c?^;=aJLSm9P9Jz2|9=imJ1P@iW7g=TM(mye6tx zG)_NXy|>d$@epbIB$V% z5f8!jXn#8uV2rMK2X?illj^i13g{d6@0qeXZjKdkSalN?I}tRDY%!g}EO>$9iosF2 z_}`#^z(7XTn71cyq(N4%6oaJmF=lO1DW-aTA4TTm%F~MG+*|0{Mp2g-wYGJUD170MJQ9!bdo3iCF>JT5vd zqvb}Sv#+YRh80xM0{FAmoG$AVEf@+9Si;VkIzGdic<~tsjpAn7`APH&N@x_>IV5?f z3y@>JV&|rZviDZ8Fsgn4TbqIJk7?Ly9MlV~r5e~%oo%@TG5c}2W>HJ30b0SD8S|E0 zf>@IGXn1*pWA);u;Q9n1IJIsVHKeCx6!@h=VJO8Ki)Ud}P7e6M3cTcP0x^gRo%xsT z?pfYJd=-D)jdyV(7J~}!3(zbqY;$tVj7vrV;0#8DC-tW`q$x<|P0_K)k;% z6Lt&Oje{!5SYy%{&3hZ_t#8*3iz!9}sNpCE{Reb%*@V-dShKbnDVc4hW&--dcfY!#L>1hI@3qEK%Z zjgjh`krvr>kOr8a@FbX;DyuH)e-%q? zigirq^-r3EU;=N-xW+<(3ry+*7Mu{Kth1}B$-6WzjI1951$?MDj6Fp5q^4+C1_`97 zx{=`_vWzq{biCv2ZSP6?P|BU&m%x8QSY5N!pQ7jii0KL@ZY(-p$Qo}yhW5dQtG=M~ z9bnMbsG50_Y0Q5Gs8pS`>vmrSJD0O&`HtdG8?}){7bBjf8ifr1Q*VbK&Dr5dRq%0T zMceLa>STWmDTwf2;<((AZ*5*9xhNDuYO|8GP&!rsV%WaRKbqxmRm0GY(K;RY+!mz4 zwWJ0vpv2Ho%02|8rYUIPI<`adg>wwHsy;^Mm!It@YgS9~2nXANtBqYhQ&qbHY^flM zb9$ql1?0$8r1{hE*xV0S_MW33;z{Wd9;3w-g-FI2+LjD7S34{k`?MVr&Qmb*skUC* zlAhvjtRT}mWLs)OO-fK@tK?wpf^dE(-PrZe!NbS)m}SkkV<%vdFC4ew52xW2-JTtX z`DiF}xFyU165d8@hJQN_>v6*X2_&m>8%$TQ|8h#Jd@gQH!4U$fXzH*ajE*^8`pP7~ zCUM%`XbWA6TdprxyBx40h=|gBgf{Og)vw`8YD$%S+RT?-GLcx@t24m32Yhcxs|`M+(lADCyOtE5vHKud*Kl6H|DwqxddKK0K{GoOphVwxFXy@nhd; z&S$xaD>5rZ>e@A4=ZgNr!M7b?TQ9u2tfFNZk{Z0JonU4?#Mc(RMZymu6cqEk8oj%GGH7EOIM<@iy)5&mIEPpxRTRa&q5)JK>4m9abuEhGHcLR-X6@bs>;|tp zj`x@Ic2M(2Ysj~klKSDgKVL7PKPI*pY*K9hp$l&b^5{o^;18h-vsZ6tKvr@3aMAqR z&zT2dP_yE9y$5C5cW>qkK|G%v5q&KeVg1}B!IeYOuCq&RJN<7SC}K_^;J@J(yslvf zIBVm8z{wxg(V~(0Kr>nX8*@E=C&l_Y5ihn!fUUYc#;NkY3=Vc+JAh+M`>%wF$pYmK zYX`?d4Pe3l+yVf>|J(uq2?_v63M-QnVRuXx02oQmMP+CzZ?jxf;jgA2BDQ9Z5lb4% zsm^CzN}Q`zxehs6d6sC2*5#$wiIl}i!h^Ai9t=g~xH*V@6In2?O{nxP@jLH0RH>vK z_f^z0BNtTjP0+jIm_8!5r7nFRD@l-gs>+sjqiD!3)1ns>wbe9~cg1v~nzJs9oYf-2 z2su>SscN-PNtX$$4Lm~cabnlWlR{3oMog?H96FDgqPrP)La_s2@sz83NY80znVD&G zpJUF~WUXjkD{YS#$>ua6gJRExA%`v#G7jg`i?)*7(_nT3E{^aH>&;B<+i3X}U^$@Ooaya&C+!aWty9Qw+P~N>yhq-s({m zL};kViB0OPB7guGN&l6SdnT`L3TL?HR9KfGCxlpaHcBm0gkm?t;zlMoMV*K!qtnU?V*l$F%eU+(!&f>%+^xSMX3fMNs`+5F6UD4V=A_FEe4_mvWs%S z$i*FVxXpTORur{WY-G{yN(%b5W=UmsB-316bFU96k8{?ArQEB`^1JM)N~W7ko8jp( zTWZYiDsI{o^Mlzo+_YLx8={Dt=`>-AieWlM6U5td9+XRvwI!34uVuWjRBVSRsd0~F zQnP8yxTx+t9%m-_WR}&YPkWnlJgBdAo*vMHj zU6kuVw!I-2T4#PJXh~AV=-VxD#Ac->ttsh5MwTn%JVFYxZ2KYOHxyttvw zI+ct74oUx3C1SY|lZ?taN@XJymhrmlX1vFxF32$wa@``@N1~Yd%*Vqa1yzaV>6UF65D3-xTRU}fmtF)ApMJLQhc zmi4js#GPa%&&8`0LQ-j}R;ctz(}z)PHPsp|6H-dY(kNosy1r9#=6N>okyd;;OfzJ9 zp|Xt@`W32H6$iN`q^^gi@WQBt#c)jSlEw{L--*tVT$0kE>!eSG$f`UU1u+wx49HEg ztE{`8aKWfqOz?&(X0eb+5^7YV&Z$97L`1Y|TmTnI8^$b~LNz-U=9SP}O4+SXk|lPc zt7|S>HF8Z(l~C-YsRhugldkF7w3{Uup*Eu-dR*+RRLnwgdM=vP4K3qymFc@B0+{y0 zT$dWTQRZpVidscM;3cp#YrRF94^6 z6kX1e`b`l4!}Gb#q65&VKCAm--}i}-WuE|atTh4fObo=<=R}bZffGUX^?RSL02Z9s zh=$V$V5Rr|^8~df9+C0(AXzAm0erdS{rP+W?6k7fLm^%-VrdIN@Nux>x7!CmL9R@C z@-hOgy;?|&n{_Ea13=MJ2?h3W|Is~jJ|`I9!Ak)85TM--bBAsZVT{i+Bqfpe*?`_d z4lq#??a2{((u|j(?i<%9z?uSiI5FzVHGE%C0AlULVUGKTDVvrvQp+sA19E}v=jFzc|ZCEPz_}nf3vB6A)tabnA#<>Z-yg(w?h7rdLLFP9?K$LKh!+~ z^b5%Gj11Zi4Pe-YbpkE#ebzL&byJ1Usk46r3D^$8%*eu0~@9fKrR%N?-eCB*Ks z^9~^3xP;)|=;0vVjsTo2O^TR5knsDEJG2%Tz!`|c0pD*iGGF5TuJ3~87Lj#F#Fi z4bY=a5!5k&q8PZy6fUmzJ)eTfa>gF{Hc;suLr?>we@r!1h^9VFmvVZpDL8S~SK?dX zSCGe5#d4}QKCs5p#lXob<@b4sx`Pnu4G6`jJiwWl$?(`l2+jsT8O2{>#H=OE+PbU; z7nCuygR4K7>j}8p>gm9BtA3mT1}xI$)h%dBOFOT(dzJ;2@A{^_GY|-2;cMKrAQD^w z_!2S@rK^XJk36v-cH|l6FA!r9i#9P3g^Jb9Sb&&m%M(4PXtC}A$L@Lf$lZ6Lnw_v! z-yu{7&!8wsL9*ixft}MBr0m?}*5wY~#-#g6zBCul!g9(hP$fCTn_tr{(&f zPy-l1+WC=1LDBwPm4Fg_=n70c&(rA~euP3nsqqz*mS`52$j5D+tWQ`Z>#0$^UlS<1 z;x?`%M;zNXLKvv=rBQ3p;6x8!z2ooT1KJhNCZ*H^6tqNvhk6q%0xctaH{TR|(F4(l zjD#Z)$gA~D0>Lp~x=$nc1m1TzgRlQ{6YCSPD!!SMhe2mB@&=zpH#kaAT-`13HNgfiQ1-k0s-%QmMJEaZS^+^6>?jkwKCXC&U=;D~ zJfV?Zqu|jK%rMnX|Mluwn9x2<#_GyxQO4~j#1KOIGKaT8iLZPN=$W!&s4SM>ankUT z&QKN8wT1AC`-7fDiZ4=V)ah6`gpWbp$w7G9;P;vIo){>xAZ#D}1^c@E*G;#OtAbs( z6~y*_03yoLb5;t>^(ebBhz{JZQq7XfEGWbXsTwh3D((=|PjQNUh;K{m@6;{O_~8QL zlp|4~9ml~9Vt!lO3R0Ltic0}0uS93fG;#r|dhZ!3ml;x3N5uy4i8%2Iu^)ysFgDj$ zlI8S*3~noA6%1{3q3?W*Vg@-p6(@FoLC(F4PvlDKLL~wbM2L{$nZZtXky&vQf5g++ zZW|Px%dy7*w`l#n?{;g$dR zZF%g_4qhrl(1RD$BiBr8L6h zVp%x2y47Ll zNt2Q(8xxop!KnhW#h>>duDJ6P+gBkZB*&&$RSTc%-Ld!q3$z@kQ!dy5OZEy?f7L$T zi3wUi=0rINgyncH@oul4X#<@SYH;<|qZC$n3{YuVS6XW)fLmrpBfp*L$;AG(J$|K#Jr_P1O_bJGq96!dI1 zZ}c4tuEHNJ(I}S*e>2+QSykY`75tS#tut~q;3$_H+yNjUyAKPT+wYdMyD8?3!_ot( zcY`Zzq9;13s5l;(2{)8vE7!qu&?~@uyM|kK48~17igp#Q5G~1U_JBJ@4uB-}!Fs1d z!Zo=2*8SUI4+bxI`1cfFuhP!O%VD+%sMmB7;cK1w=O2QQgW82)f?#b-Di?H3oQ~k| zcGFylV0T`KoGC~z2oB!2u;9<8TQ*>^!N+bOglUbiMQ}{w4&IpO`qEpU2u@m5p66({e5gnQ$Sq_ju1&|rds zohqbbqLJ|UY+vZI&b zi|l*g6N1hF@Uz~U*%MvRjxM=p$VcBeE|B{ZQ5Z-LQSICRIGODDa)+cXwsQ0od z@>=3zD+5xFjW;E&f?O0Vxb}=oWuQC1n6eS`kYp6y<2i<8<(h`Vv(zdSaVaJbVm@^g ziJ6nhW$<^uT|?_O=W=vU;5~hjhb0`Rd3byuw%sI*HP?Halqj>znD+u;3nYNu6KKtwx0^}SBblfeU2QH0U!Yta1WGW&)$4Kef@=ytYT?2~bF;|RUqk`nY9 zed4Hp+UWDRBh+3$Iad{OLG1vkcLg>7NbCA;@!F>temh;}1azLY@aA4nN6iD)kHkma z*ZKxqIO`1cH~cn=r2V2ktEOx7qS{sf<9ujM)=MxMrMXib12}Z(?7y+9LV(65hzuxw z@B@=Ua<#Oe7iQ4|?`XkySoA8Hxv=O&t zd$^#39s<}%`@l)F`?r`zuA?Knw~MoIAj8g4kyDAm*((5?ei4D-OQL&XW6<+>t!b@; zA?B5d3w6JDFNVPak_jlo9n%@n5knW-crVh?WS`?AO`8r|S7T`}d}pluXOt7c641{; zTQFuQYcbjfNBSTgX8eX(GYmIixEO2f%-fdsF01bY zZUDyi=Zb&iDC-qtFHN33_K?>H*%jEBU>u|ExyBevF)i?Az__#p?J(|qpj9g3@~_4- zN}Jd1r2Kg9%$N|fbRt~1r%xn(mB|I>zu7MQ2ik*+yurJRBAh*y%8VkR+ZMKdU69WCl|R2vwpeek9V@fHT#C^nb>-N z8Kn##j^JFbkLwJi-_@FD_g-NBcz^cgamK54qB5OepkiK2C4sqz9Go0G7FY(`$5%51 zvVWo{-G;@DMO?!CXPdSQa%8NUBraeD3pEPKMeegA;FLD}C+_i?<2TdUD~Pb7Ky<~5 zOJFe~B)wayxv8pJS1&T_E)ZDFGxg0d;3Y`jAB#28@`UsvrLi2_|@||H?)hu z<~-ulivlg|3$EwujP0)GfLR3|FUlmqFU5|w_f+c18~l8OuM|khmO*EiDP&vO-h-_@A<9$U{h-@&eM=_gnD)0bQpa`TH_ZD9bruTgF$v<$~7*ay3f z!0}OWa&=4WsZ_&0)pFDHeCxEC&$}y31^Z`p8AQi%i1N4Vq|l2#lwD8(Za9L#7C7RK z^QytJ9XPTfZsw)MvGwx4*yfRpvhqWkYI#6$edR7YaQxl$7SmEV⩔Df?{gZwi2L_ z!cf=1mqd&6*$-rFeC9qo7)hlA1H0qysLNO_)97>aY*$V@HpB7x%Ar-|z(Qt<{ zrdZDRi^PK8>e7SvJ3+mPI~ucwZiO~jPjYjOAMR$q9eDr``I!UajBUiI@Zg8OqBO8a zi~j3Rj+jJs!V^k7Sk!M+_UT6ic$&Fn6##j=#yfSQ{ZOEW@hp>GOCGl{zFZJ~Q#`*| zB4)hb?s$n|$3c!qnqEUO-Yc=%MB+vdvYe9!AN*5@ zP(sP;E#;d4rmJG?9!yRgURQS^fXi_ANq$mo8Ikyo8PEy3OGqFXPM5}k?I(~}firp6 zq*k|Itl1RI@Q~NX`e4Z8{>p!5@!(0M3L^iGKe7wKqL)yUz8bdrSih_?X|Cax6I}26 zV~Qsgi$kFENr1E91h-t3q1MtAO^&N{>n@Pid|BjzPQO*sE9AdgQ^qh}K&O@~_XuTA z4jGf->l@QUA@q*H%g(9feY(JNY&!D%8${>^Vg5#^cX(AT!Xk{8Ychp0A5V}vfarjO zSFX>5M^7GKB*%)!gA6M841~u$ddXmnY`I1v>aI3&qM-S0JrDM5su&T)lW4nmD{Zp6 z{Sl>Mg`htG6kYf()%CAMGzC+-u)%*jhDOvcmUYA>flV!WbCO!; zFOw%&)S3E~;ww@30>w7(X=jbo4nQVI+J?<{5|BKE6IvE|lmkDM6?d^z*&K}cF!k_E z$m4asHRB+&oXB965EWsP@%vqpa?402et>P0W>T4U6mtWFWZAe&ZjcgR8iT{2%^jkX zQ4aMa%TGx1XG^HqqbDHw*7D0mJz;n~SK{RRfz!`#jXaFR zDWnKyOa5S_bWlQ4NL0}=hHvq2AZ6R)-!o7csrcI^RNbfy;`;-1qt`WnQf;_}$dD!`sytN}C%HGa zAH(-^R;~j6$YSu8li9ed&AD3mQ*2j$Qpiengt7npfcn7#vR<=n#Z;kC>Zjyj{M*F^=ztz0z&g*}l4 zzivRO_yEZ#^ft0vpi)v&nt1kEwmzlL@=QE_U28$vd$5uJtjuYp98t}>q`W*Uj2Fxo zFJnbijLUtXg-I2|mblEd>P?l%Z{6}E%$LG6DKI*M?v(Hif1uRPHBes*rh1uSekONn zmfuh_4|)n9G2nk%a^d!o#w5pQuQYn9C(DKUJ3Lf+`mo@aFCMj|C0zxTX%P&E#P>lk zdX?cByokn9jHM8g6{tg>Fxb^DP>+ps_6>vAuZ6m2ZbBfx1T(PD9mwTYw{pcba&U0=4i$gc2IqAO8?*A zIS$R;hmfPVkq5<}!pATv%@3vds8?t}7YWnfaE}DaG4E;dZ6Fi$O~hS>zSfOd^rLSc z8Axk0`qmN91*{pA@o7$~%gvJ1O5e3&N-d&o##PQ0nBbLz6+hd$8QK3r$|i8G{6@PX z`$3%Ui1{Uax?e6QN%-XBaejD7`u{?R{%+ER=up6D7Z`sCJ{mrpFCif;G@JpN8uczHD%qFag2{3?{Q+TN-&rDKuO16%8b!!L=WnKA=& zPpJIR7|77EM0Zj(t5gS#p?7+IF!LjlpsZn2lqCc2!;(Rrn8$!e>CWS|kmr4Y6)@>5 zqYVDPnR#44R3|_ChndjiAlEX$t)3jw?vd}=gL2BGtB&T$0f0J2L`lGR!Wp^-k9?J8 zY{-1Fy9p&5W%F<=aAKUXo(My%PALF0L@~8K1BbyAaIo zzRDpx?o32XIYGN*Xk{Wdr_Q%IQ~f;((MFjXeuE=>b)nMais&R2QR>Omqh-Mp%A24RQJsiUBQ*C(*@j+N^BpWWPMmEzh&3O7E+z&G+0kOKZgSNmG+Fd~!L#x15S3dWl{ZO; z7Da)68zCz$bqk-Qk@d-DDIoG8(}!=x>0>fTvHFaq9q0YEujlUi?O4W-COcTxByX2J zYZILp8|kcI08NK%3(FxJtaDb*ug}${-&1;L6;{SbfRI(G0(D=xv$O?*z1S?m0Ld=9 ziJ;IhYf!eGN6GFo@=EW2UC50)8haZ@_xXjNE-m*a_D<}bCL{rEesiC5C^r(C z)3*DZ@x|f2&rNsY5G7`HUyzJ*G8E;6DD6}HBo<%3jPLFJBFzlz9L9IR@kH(pVV)&{ z9d38Wzl?}Z964Qj+c^4s8>z5V9yG@m7dEp>j-M6>;EiPI#SaPTSK2>gPgkIGVYX~> z-mndFGIox6T8ix@Wd=Wu7%+%iAtc3PY0;rmb7~Ws5>~+Qlg$Iit0`j>t?Y#}IK)kM zE}BpwlS#lOAoW#B&~;TL+nm0w>_q0*vWI)lWK&WVhcaY*z@>YZFt`F4zuNC9zA*Yv zC2qwzq%1SDW8a`6bk0pmBF>#D{WbhWuhn$j5EpKiz5Jy8CE{grak!UW=eIq6Oub16_o2*HWXHKZ3jZ zVTJ4S@rmvR3fYF>$h{wg?}YpC@^m!!7@qt#fFcr}JoLMMnl8Ct1@m5Z1(q@Do&oxq zz5kcmJj{0o@zL;UKeQg;(l=paEPM;&*{oP{<b%=D7!v8bE=Rl?Z?B(&!r{t7 zxGDSdJo#C?}HH}*={HY7qNCgC3}Q!jmX(O9;bp=u7Qfage~dpox{8~EOVA)%L|odt67eeM%E z7*@pE3%e9YCNb;6oYyoK(hQb2hwVwQj!|a0HW?l;5>m>1)}8vx?*sVA-x5Ig z@=1iE?ACq4rM;MkG`w-gExq`*CVj7=51{3<#pqS@k$|Byud=W-hxxy@PEd-jhde2GSY&^wRapDNrnA_k5 z(OZMUHq<}^QWG8FfU4zrAROuOI>s3+)(M1N&4hEm0BrAhj$VTaVf+vmB16j8DePQa zl01UVo7;jpoe)=-LEwhbtq|AQHU{AWC^@S-w{M3+6Xn9y3D~5Z#Tg!^NBxzN4D-O2 z8{xM6@urUOBGh8CFSQ}OH$GkH5*-`FUr7h(S6U*(6%*lUI#+`v607`N6I;Yzr4F-3 zq^zeVtJOfH>yXdY?9b2Te0C((Jmw8nmC&mkxSnZ zPZfb{l(6%=0$YFE=EL^H^W+7(BAx>UF}&VI_3iKCMLy4D^*g%Bu@Uu6C7NQ3B|}-U zoNy{$v1F=7^zmj_I2N&itGVjE1VBPHr;jmyzv*|D6;XCk=p|#q3H6G zeHul(oJX!TLi{8gOAijbnV3Bhzhg>YtnNkM^q{`KYlwd08y}C8=)3!mEnm@(UDNv% zGvh+S>2omxI#)&{28kM}X?n_6$e=jnUn&(#KIWJSG;2T`?{H(`B&ONCT3u_lwZfr? z;`(dV#$Q+H*U2@0FPl;?=@ zRlb%_4!kHG$1*Nt2MxdRp}fZd%M878iXJ~nWJkBfR%raLdxP~na(ZVLv<54Y+2?V` z>_pDfNX=2^vQ!`rzOfh@S@0@va3&>7D+}dI^~A&_%R`imSW{60Ww8og%CbcAFY{Pk zfMQ0umg^S(=CT5z3LQ!1$qTFi8DmPuhRIs+p0KRlD5YD{-9o&2viVjik{tjH0ptj! zT)u%Jd&)Qd+aRJ$+z5peBd$*6N0!Z=EC6`}1^vOlklBR2u>7c|OpLgMfAUKRlH0Ma z$}g4TI;RxCAG`zLf8L;!ZBKqp$_>?onHlRKau&{*eq{2;2N|{*muJMN@s#JlE!#J+ zkn{rbE|6K-^UKk=hS*9bpv(Kg?~!~0#)PUsxM#b750v-gr|Oc)_<{ct zm7zB|$(kg3H^OhwQrH*sZk2(! zncwpR%6CX_%nCWH3bp(7a;i>y)?oGIHv0m^fCkdP9;jk8#8XvY{Egr|Xt4E18CfLW z=)m<+Qrw}&8r;M0J%ByMtxPqHGF~!-B_1i`C8bY;G%}eujOuytKY0Jfuw21HH1eBX zE2xHyNx!pdwQJN})FZ*BUe^?h0QvMZWpp(vq)+t_-KBlalg?R-D4y=X6>pgT#?{~~ zzlKapbhDH&++JF8Xsed<&71(3L<+0XN3_%n04%L?W>>3TtGa-nC?NSET5GK^J*>S4 z32wn%>;1yo9&dxC*jE{~byN70`_?|=LG0Vlt9u5ye6Jxw^{fPU9e^4eTnEx1R#g-R(pjmaF%LT(E60bw?2$hXs8bx>5;dHEl@7B; zr@rY@j%S^b2wZY0CbBp91$shSz;z}{P3Psq+Tg=Fl)EleHt2G#8oFvtKxpGF`ToXE z5klc}t!rWAEANj)b$hbT*7b>e`S8Pq?H90FcdJz#Y18DWA&o2AJE!GC_nwe2Jvk=& zB8RFM>fd`;Rg-78-*q_EdQ0@e28`VIfzvxDwO)cJ=l92D$L)R6@Aq>Wdr;_)MT%5z z>#fUM{S`ix-<)5g@?!O%_XcF|%X-dEzlSa&x&kbH%M$+&VEpF;PyglZ7ZS27p#ezA z0|Sx~(z^fcK55n7;YXpeXM=H>0AF+;BS!-d^5Y8pB(=shP6xUG1W4LzuS zh5@SnW+a{B^3kXs4}7KTmmxb(j^)FgSbzMC=1%P^4sIzLVMM{KISGGeFZtuXjmPln z?4c|BjGl|3A!7l?3)IwjeJx(biGEEzq|t&=R9r}Zw-_jGhA;?{{$Y zW`>g({!BY1(+AwW2gUP;?~_?#(J=2rwo#)SHmevT@vNv>@9P?xO+GU7PF@lIT;TN! zBJ*CI_feN%i}`hM z-pa1Z8EgMgX2JOnTL?qzomrUGH^^hh0|-;dpVQN#)6zF@9{QUQIl+QdhHjmgHT(Axq&scTmRSThehmIBb5x`)DcR zaa!h%Gr%JizjJW2>U>d&<>r6($--E*+$v<1Qfp{xXzO(l?`G)IB5>vHpnPbnfe&$ZCtI#&qmohRJviZ37jpE zsiMYZFqEoI$^`Vl7VJNnO0XYHo4LGTGZF?dHDWM=+%O- zvJkS>lzp1N@13L8;(yhXtnLC|wuT&L*%!3}>f1pC^`;jR^G3e9J^%YKJ9WX|k-x?M zqBbMJv4MMJukf?6^7C-X!DrjBi^<+n$s|t%Of zEj|Fp(FO?O$6J}$-g-x*LQCx1lz&rd4Y)+upH`Sa79>Xs%J za~ih9GUf1DU1)T2R4!Pfm;O%R@|!d2S6ON7&)dzZhk6meV`u7e#c1|l%q-L@Q4I~L za4yPSt)B~6=E2)b#zb*WlnK@^hm>T@g);q$Dqw_$oS;JA5wGh~$iOce<3mUA?DvH|Va)NATBO znAlx!f1t^E>8@={7-pL(C^?myaE>lkEiD!&8}V|=9(8G ziEd}J*E4bx=0(NK6HxSL*py!MS}8xj{a-DVGRJg_yqG+etG%k5|0aH}2<#<{jSEXJ z3Ix*Ky9l|yxp@1E>{WBphbjZk2E zbGlsfH7|eHn4;`{^OlZWrH>K#>ZnY>?rrDgJl;>`yunxm8OzLyA@BG7v3l`=M1&6p zs&l46{J6!o>_ZJWNt0@~H~ZF8k3|$g=2L>8V}YezR(!!E_xaUDeJEjOC=IH80oe1m zY%}qd3wC_s8+HG?Pri+kT7pZnp=(VT`_AxfeAi=Gr7eG_D1PgqA5h8|_GO}d^X&4| z1oOok{MZxh;d_1t$$ZlxDLe#gko22_4G8^gcKb(oamvlV!uMB`rm~q@O z7lB`R7eq6P_{dJhw7`dLJi4A$Y_{5X=#XUJFh71jC31;&p&@@nY*9;r@M7lhr-p#P%lc|&0Z18!gjKC6#DsSPjp_=(6Fi(tyZ7cw#cN;?XZS1^A* zL<_!31z*=6=dB^ZC*v;jLG1+ zUlfvhB2Wju5q+nP${}q&+J>2($TdtVd^?gSa5iG=Fa`1>kU1DPore`v0ctn_mq89VALuwOtLNq-@i z3PlK&(EMUJu&Y6Ym@?oz9`1d{@6f`!!sVMZj-e=vPyGDK^g%-hGI_VbyIRhTJa{IB zsoYT(V>jV0#OqZ@@?F2k9R|vF;<%^ly^ZKuBkF_=gLcj;P}sj_F^KT9a;|^ZP?2Z7F-7~G92O4d zb3cB~#$XW+PlU9kP0DW3xN)Px4X^sgR;fDBJKXlTdYR$ADRjH7%D7^XWNvr}>BL;R zM8)7Lvd0sLCw@V#P0^4(;CEo%#1|W28lIe;)922^tBoHFnouXd0U6iuzWS;`IlI4C3j zb_v=3f=GO$??44YRX;__K|?lyjkeS-D`?Akq~6NUV24ScN|BfH45M~5%1;zP;-pcG zF}@2+?V~Eda%D;M1v#qzHZk=2Z;EO{O1ZRP)MxV6hQmp5=d*D|d$j%Czf93*0i&H5 z%K#flHET9qYrpYAu;>&F4@i%r81U?z>=Dh&F~b;qgqj!jf2+-oOteUv_rEqqH(M3*jCi?2eE=)`wr3J!xGfy@R}2d^qE{6mH-B$pES{Za zL4L7*l2r`0e`Ab~A;|i$I@PV!01+g8mzi?~=2!6@CIzQ`+=+7S2OeKgajc zEZ0e#`w-8`pLdSvYD34m@<-zLYXaOJo;b=DJuW8TF4Tvik&*`jfyt zuyT?h;YL_Ku30Vrq{Z*v?IqJDUlu$kvTszx=n(lCIvKwC6e%)PZ!axhNnp$`K=g51)p9U(wzpdU* z7Om7sgmC4a`ls*c8`|p{GZq#WK$le!Zmd356`GcetkC^GY%Q~q3&{2fM@JtvJzfTv#6eKl@IyrgAMqJhl+ z^3DjKbL#_;ywY}NIDXg{HP;O5FAgsiejI?(AnE&LN=)_=q)qN+n{eFjKY|zswL`9I9isox0h{rqsc7-VKqYP8|^9RZ% z)xZ6brY^C_4s$1^TI>Paa#8BD_Tqrfd?Wes!4|*aN0El1w5Q1}+Aj!JZcdkW?Ej~= zt3rt>{eE4naSG@u^^bIe#3~KVp(r_Ny3s#mz0K+yy&9ZuE+^A1{b{ z3O_RPQrchp0L_W~2qBj|)qDGSvkbJ@TE1Qv02%B|p=Qy1;|y}U%Mf09qamZM7GY%Q z2c8GFGisu@Gk>lgx5iM`uBtnpc?6*Ky$%ur1jCFSXyf}etE^XO#!iomj1$tB<@KVO zk?}-9~Dd^(tGd^1#w>ekL`eFb2TSs)_Wus?-!T-`rSexi; zG>O;L>sQ?djD}T`?c4h9(Lq_xBLezYOA0V&$A1ERUBz(iPJUM3$A~!UUa$Cb=wm_=*E7}Vi*C8hoVFOuV?)PHMLljLf1@~g@BN8Wq zJmUJ3r``Rc9WrQ=Sp~q@x~oZG$kY4*Z^D&2<#VgO=5L__ppI2b zrF{Ot;Pz!Hb>J((`S1CM*5SJVg4e0L<25Tlnpw`}Yv6zufg^f31-k}_6s%|OSrnXK zqxA&N;2;2sJ6m<}w!~U+qiqz&dPp=aoSD5dsJ^{Ip;T(GphU4$Ei@`L4~IKs z)~7Cv6_TG`P%x2CI@y0GDlGV5zaQ^NCU??*17aiP)Gut-P+GY3>hid^3!f26YUdZ| zB?s4~+QGPIO01vcNwmn@sCKqq%5(=X^FFEvk@j`*3*2Hgla=_<1Gm7UH`*&KS-=( zkY(F2NLV-PvEpQK>`^UW3eaO4Z!OQ~P_(Wi+8DA=y<6g9T_rjHoH2 z??@-LOr8~D-HVY1k}}ItPGx2tCsIJ)DrI@KdU5zaAJ@^BI{wsr*~H2!SR{I+D-fLp zg*qB9%w6H1=!d>nH^o*{vBHHXoYDX1*)T zM7c9c{H>4iBP;)uf+(Z>WViw4FG<643ycc5Q}Y*l<$TDbg1Kh!Y)>!H;&}#$@GCe} zo^`*@#s6XYn8k!RV7Rnj7cPpSt*(;{7^pQaS7d=!MF}cN_Hi4%5n6CF`F+W>M@184 zA#XyOf9q9HG4L8WpyQ;BX7&I~K(oL5fBji2 zh|tRY*zoEd5M6-?a8*T;Z$7F7zVVCEuWFb?2WFMNTxPnj!`X`1HAoT#f>ya#OPsl; zimJSQN8e1SrUk0}Yz6x80G3sWkPhVlRh2<`B#geg^9fY9`!g2R{k0(fuXbdf}~F-`CuP>$S-JyYTXCSRd^86)r zgijN_H>cIw|}TJ02P_XlBz%$gu4fBVkDu_!_K!#Uq68+&@! z+HNU={1QJ_U#Bs$GF^Z9?fjIxFwt)Q3Q>}9wz-qj@>g&6J^LA2NWu#hkKostZ{m?* z)(cxzJS0dk{~gKaH)Vt&^4s8W)-R+W7(wmIeE>qjnkWKsL#cWid0uLc5NxNQ9=tR{ zh(YY;BFRXEIDHhiOq^AOF(Ua3p}!Me$r8~<1Me_m_N=xa9{hzFe?%NMP>?y~g2xL5 zO;@*iv`3t--3Kb=HBzX*U>IF@z+cl9^8Q%v2J{wv z?@)I#7e^_Y!`TVm^^@&U$^y!&JXXsR-O4peV}bTB?TaY$)Y<+;1>TpNh8{z-i#hfWmTvqRxM{A8AI`+K=#e8`sdhPOLQOkU0z2&}XtHf8B;BOc53niiQ?-tu+$B5xFO+Y8xO)PKtJsT1GCCetAef~s6p>U4)1%#Qk z`5neDkyZ;{@8UAtmOa@)I{6$Kvnpia1%S-XbNN zS!E~=Sx6fP0f^I>2_`1&VfAuXcdysP2vaMbIL+>CC+0`2?=tA(sruXC>m|_2Ay@8W zd@i$yx6z4sd{oB$fnQ34Km8zh85Jc+09HUlFM9%B_LILIAE-YUlPeO>3??Xh62#Yg z>1#Ea8k(Sa4wx(_@mX_qeoxBl1i#?i-_6gE5J-W|Fn|dOR8IGV6y~mZ9QrE`UauS;gu&M#J3^xNB-B-1BHAf#L#V?h+af2Fpx_) zD1n(MUlceiNK|{=k}KR1D8jhGbaU||x!OsXXqf%L%G?Ky(3Mbg&Z1a@$F= zac{R0qeV}*4{%~mqNzaQ=?|0yl!7--orGQr6dVo=I81&TQsJikhNTvXw9r%TD zGU3bGDaX~{qV+~5Mw2D!7B5-GVuH)_v8mM5Zpi^K%;}<5?F#T#R+>D@tP7-D%}gFRJ)_K0wSfKuRl$G+Kwk41WIYS8g$r@ z2u^w-=^w$jH``K!t!OxJ!&MQGrsk4V`0+{|c0KQz-ld)emXoqN^@GM5<<5LBTx8cP zAJae`O-lRpCt4iU9x;b>yi8Y0sP3N+lFqo?ltd}u_tQA(Ql0MR@3^7%b(=B#q&s*! zuk>hPozz#tD%lNeRd(3U&#ZnB1*(MppJZd{LNt=!Nr>tgC0e~^hI z8_x$I31D6}K?6MJ+bb%|w#vaRIXQOa%HAGn@G}5Sa^U~qYXmu>mk2Abn}^tY#0F0G zca9GDhjki0^`x%_p~!J=IsQ&hb*;LtVhvS_9519|fItL4H=dJMHWNFQ8<~@zJo>mg zBN4wN%Gp-^2C8PGopXioI<2cCyX2z2yAH(HJLNe%bH&+ zd7$B1-U)sK<2`N9dKyBgoDu3^uPr6?LV$kXi=B$9Aua&%jrZPEyPpOxR$hXxY z3>1iSQeeIn=sjs-EttinsfeWD5#9m*-JAGfeBf#}(~c-F3gIIaGjvftjVYZHD_^!A znN#8!il=I`s(1!25W=YAqA(2u*ZS9HnJ8>*v#9WxEv|nrQXzR;@DFWOa|c85!a|@; z+Gz121~f*zUUQCh^KUG|d*Wlc{Ma5pPyBcf{;f!k8jXy@U@g)@yB;r6k@6>g=bTVRRL|t?qIx?gkT0t58|aH&V_J`yV+GKsd@q7 z#@+BzGV0gYLBl24q?kS_ptV(UC0tuc0iuij^DOxl==IRJc9^w>$V650h0{M4x_$VN zE(R{hE(LfKR_q=cV1-J#-k}LJGJQ(rZzED@4GsCv&y#8iD^;=n5wrY5%Ah=)cCh&}P`tjhAKNQBu}S`&yLVugu!Z5mdU6_Q$1s`jK@!SmjqRr_J0d6&W zS#d3%M6H$|;7LNnYviYDarn=6emGkobLt(lG#SaEc&N{)#XEET9H|QVoTU z$hU@OooVdX(Y0zzDQ^-v{@6p18tLCN0UDEio{TGhs(USmt3J0I*$@H&!hW^gV0W*v zs_a>!-fP<7wPvpmrJuP}a3l^Fu$mnp8@NuyINUkfYc9&Xana zcDkW}${T;{_XNN%rw|5Ro#jmYchCaq6inH=72rj4YPn&})G^j_DAzJ}nK8acL~ zDL$=lT}YetFAlG3CXEhKUlvl-{WFL4T(takvm0<-5b$f1L$uIO1AX7}xsXrZAf8>= z-qXl#U-XAcf<=Q8d`jE^oqJ@NHjE7#-}IF=bj0N5((S8Yc2`{a`u`$M!yyp!BmYiS zqtF>7smj6|&s#rMAL`n=bY}Sep!m9a3pzLh+ff3Yry4a|=xv56Gsdy=pxX7u0SqI2U4Ax7LMfE2@P(Zq}2wH_*$uX^p%d z$xZCeV|`W^|4v@5C+PQJyGjBy(otzWb6G~Fs4vxeWteC$6hMd2_7f{(-+uDu5RS~m zFU=!V35wWnV@AHMM00-s1iM5vCrc&6wB^a(?2#@$*c1c-Kw4Wxjg~6y@E!J0S{h&y zXq#6%%BQL8B2cZL`uyhYh(Qw^7pZDC892?&P*x@l+R>=p5KRdt43?uvyn?cJ{KiYz z&MNNE#^L9b&6xkBIgx+pn`b}kyO1h)neo3BXl-A5mrBSzM{FhH1;#K(T+H?8c z?cCZsFe{76pg(rV=F9Y<*y!Zb{E=Tr*s(2Rsg)eu_{VRF-ckFpOeDb9??@GKaPz>p z;}oz3YortCc2q*?1dWqUtUD|^h5RLbig?k_TmRMZJ6Su`LqMkf?lKaZCpucZ2P~wZ zb`Md%yQho)R!38k`9_B7euZ3gdPca(lI4fN*;#e6<_W7i>tGNoY?2fE8#-6?ZLoki z8`9v~%wFGAyRCJ8HL&}viAxZxB|xGSPcVV*b?G?ocA~#f0`Jd?3EM2HE*#a6nHIdM z9}Se3irZ zZwUUrki+rq!Y>xoA7KLD!RvnRbNRVzkHZw$S(FSDoYM*oABp9%>AZy8{3 z%M)Gv+nKkA`1JHNy7u`)dek$xAXWLuu);pnRI=}bl2u`-=NG`PC>IiAuTq$>=%t!| zc8&Ex4gT_?)2^mIi=ZlK{S(`mP*Vs-;ViAvqf%p7~!GMHE#aE`l zj7Q_pC3Q9!y&sgned{td44-~*P)a*F+3P-dZIR6eg?d0pAH&QCbol5F9U_6PC$64k zkmjw(Q4mbPr*kW8G{h&YFMpPDO1lL^a^XjmH<1@G$|N#b9T*eFEWUH3C+GMV*KaeNc|-+kh!Wh2m8|xhr%cuNT$K)w9XX zXSkebTJAVwjJ*;~++S+9O>A@m_V9{%Le4w*G4 zD0-hu@H+1|^HTH4HQ_23p`BiQCvkh&6SCo?+qKEZZhC)|idfLR;3PKrZ;MVTo-_}a4J6Y6fyXcDnc)f0jCiVceQ?m z5R?h>?%_M-{4>qiA&WAtJ~yVd-=*|@4tcek+e|Y2!Y^02eAA}bu-p_}RsXIDr*hk4 zns$BDWnNt$qLmq+Sq2v7Mbzq&qY%zep?}!v#wk1{AYNxUnC6T)AJL41?%Jp`wLict z{-d&h>%KKL^inX>`Sb`%ioZ}GvMw;_-(VKknM+R2CbOnuYiIFFbmsoOBdzCPRxA4( z&KaApJVR8eu76{T(R}y9pFTfyfpDS2pIFGmpk-;78xLmPa{Jkkaf&3ysIEPs3}d>^ zW^Ps1Y`JzVSu8AZ)Vh9U_jB0E)HdfXt3Kr<75*~(|B;E$75{$rxw`sTjZ;`-JXarA zlI|=?y+ypH^K_M@L~jO1z;;W~oSe_|d~SjtIej6(VyX8g*|~t4ARb_wB2|%jtXsgL zNTduNXn~Z@aKUbr&jhmy%y{;Dyx>{T1ffD3;PwZ5k+Y=FC0k8`tj*^ z${LStxp1Nk3qOJX*l1Kgump-1Y49TOu0ID5FXGdB?}io2AMmC79Ux{1TYn5cm%sey z>^}Uw{E-jT%*f|&kuxHRz6G`4G|y1d48@EI)TnpMrDhS`M-H=Lc{8Tw6=LZ6f>OU8 zm!&SzO?vO6=8wZt-=q$Dzl}a{jyWWrHU+fcix* zuEA_M)Giet0h8+!xlnnL*9z!;^NtJpt&qOL_dzj|!eqQBczx=Qvl8)9_;p3$Mz37x z?-huZ0Gq5tUrEmJJyT;ZzY=oG8*RCPl&={3>#>T#4mv(+E9cMwha8VT@RM}JL*@^#~X zU>{a@XSSUPbCq#jv{|<`oxssGw=hH5buB?OL#RNnmJCS0=(1KM5o={E!9~5V9ddXh zLw-U2@^_tMp>^c3IO_yK+v2~ClD$ZrpeFlYx7w)T*>|Ji@^jX^pioL;NHJyoOyEYFF5I&xbeS}v zbe9Ku7$?wXU1i&sjS=S5y^{ZC9?P6j9T``*k8yggnxW&P`% zzVZ5^UcY($eCr;eYJA&%KrnBw+^oKv@3mxmPe-n7@dFUN<%AK95xDTthmj$7tt;N| zt;iea0N}Wk35^B$4WD5vI^3>5+CH|PYhfQKGN{1`ahS@SxQ({6sqD5Yg@LvmC)zmI z?)jlQ-u8&jzYC+3-1U<$NldSIp^w|yiQvcf%ytR>X#Hb5`-sXJ7x=75Rk}s81A?|Y z`oO12F-ZyQ!SdY!=Y(tr+4$+dfzXH)xE&amf?qB)GV`y5h8^?+eY{y+5k?m{Wn5C& zeaF8c@0!tME!mlN>kCX}U|_+E!9Tf)5=uWk*Z(H)-Zj{a5BlmTfC=1h!}R6O{@{1E z)Xx5#Q%Po4z203!BekZoD`NLOL8t}ymu`7I zsKM+F`^%9Dbq_DK?b0~i6J9&l#UECaEg9W21EgTwa}(%B-nj=9B$Ir)H_E~#Lm*@U zWI&hb1i&7pdxylOS4_1E3E2nc@_nD}B$GCTKPik4a4FTxSlE_~VB&z!i=%qbKOEq~DL5qG4-hD|yK8I*S*d7wc%}g9u99x)H))*YNVSoR}HUNnQ;CVP9t#yv! zVMI(A43fj`=*?Bb!%v-*1^9ggg)3tzJKmwc567oNZ($rEHa!6WwVi}h4lf}+ zVojBn!-@*BltCEa_CEGyga_sUG-Ch(~3q7O2wBL1jqA9Gqd!L;hqlJ5$0Q)|a6Jx#1 zmk&O%q96)v)SWmqzLF%GvOLLzFPKo2vXk0JElD6f|Cmp?q9MyXwe0T+WJ2-IyDxLU zK?s(}jJab=SAYR-F4c&xr=bekOi=nhLjB9@PaphB_r8*h#+~i+Psv}lIQ@n#D`PE3 zspPHm_s2eAi`+9_-Jib7c<{PCdU*$IWL7_1&S7e2WK4a3&c6VKErc0|)Bq1mr*%Q+ z0tAq^wmmHpaps)|W`XjOh;nKF0Z;${`@f&{|Nr|y{2qwE1MzPl{tLwa*#7^o2hup@ z>;G5(uVo-rH{*W*1Qq~gnq7)2TM~u4kfN|fL`3YO5MO!{cL;Yv=M1TdG4#`Y?MaA; zh!lm`A|f8C*2wj(*q(k!G}E1X$A7Gd@W+4S&)fU`db?lq`!$X~f35D<;eMU(_d`9u zU%yWGYkj{q_xsuX^Q-&)*LU~Z?d5(=?%#L!Yk$A4+GE~5&$Wl{tj9Z#_1xb+?$`8w zf9c-MwMVUa+z)h3x;=m0YE7Oo5si;Cx1#^h(Rq6njW^LU6b(D=|EV<>@%Js6ylUN6 z^u6lLoAx_)&vW08P9FL0TPLG@J`>Gr$*U*%iqCxZ-tnID>xs{?WGcFnZsfa>49?x! zyuXTu@7gcF4|UF`^w!g}ukMW7sPxVLxn?RIuSCOMYox>6Z!3Bp_silv-@8h-XYqQ| zZ!e;4rDr3_7P_Q^GMU`T(-WI-F-LrN9`^DNEvP#JKvW4b067YUj5Q>uHF1*Z(ow_MtYix zr}aM`3x8sQdmY`A$;#YQ<$K-tv}PprqR!oKs+@~-g^4uhQ)BY)GFOSodv3%I<_R6| z691=I$LuG~Coa&_diZ&hSPl~VL414`t$EI;ej7_XU!wW(&R-JisiQB|Fl$`i^(-&WOMmAoaVd`8%OYu;7(LPt&sl0Tt3W%wn1 z_8a+LMB`_(ZY4RaB&!$M*idfVZ)jGCNxi&EZm-(=sC@|)N6FHXn#_j{bHr>V_ocxtsZpFNj!{1`&nxhDpGd}y(zCvV*|)#+vps7n@YDkJ-bDP zlMg-r)`iYVfum-ROCvw|YSJT{Nk2fIO38T+sr6ejyXwrPbb1!g?~*P0eGfwv>r=Lu zJS$Z?-IGe!vL*bhuRWI~>-_CQ`|QNuQZik4_TX^K`{$^5w2n&cJJ}EE0D4{<-oGTv zTQ~;geq)_A*4Z2BP(H!={#`LH^)!>Nez$4aUDAqeAH%5zKXt4hm+d5df)k{+Jf~M@ zCeKUR`LU7ru=9^xvlCt0l7ZtOx)JO$1X$@kVq(AHZ?6kD^c_zzHR&WpBmbdn2Ok?~ zU*Ma3-cUyn%aB1nW7^I|U&?VH8GTCL1D(BTcu;<{k8H&EBpi41%tcfVn0N9{eJKwZ zo=w8JsPp?D|CAH}!uv2l&yz28s-?Bjx}$akkbY?GL{cXrbAO78uKh;RWQDD$x*440 zk!uNaBLh8!lWxAYE;7^T!F^!$t;iz5g5P?L6686mUQmLyzzI;`$C(s3ZH)6Z z3U3Z%@bf4PcZ~`F7zT-!AghFp*z&wlI0>W@Q6|nqau__X!^rd8eH50Kk>@jN5e3?m z*HE8>#(@-Et*)(1VAAlhiV9fB=T&QP#&cAia+ZvQ<@23^o`ZKu#G@#%@!ssMWM}ya z*yKf!z|3!rJqg@`*TLdx&MWP6l1u@Bi!kngs$B?ZINMls5@c0Nq%F*%YOg_K3&*J1 zNp*AT5r8)gCv=1IHbG(!0I>(Je}zFfz&-c8Xbk~SK*_+p^}Tf%IO9)*hXdwvsU*ZyuFAsHIiYIN_kjeXU`?l*>u-+?W470UhdO*|GsTyesLF zcyj%sleOiqMC8B>pr-G!3KvICyZ}7>`x;e`j>E~)Nv8F)k0>BTk_A90S(#5II@i(g zMp(aulaKX;oC)i14ZnbB>}(lTtG9*Gp>${P2k`ocN?xnq_)vjoAEJ+-f(`pJ&_0xY zlZT`1;T%=eP=u;*%->g0bw2u9M->F>z2x=Q(9-~2?1lo)xj}wy{i&zrydltGaC(gD z>GaBulHOSum|1U%uTMn}@)^swx8Zp1ctN3s%;~{k-{K7!S(nQ&utRpa$D*-CVh{Qi z(01WK?`lzccCQLx*e`p{b21nI__rk5mgFiuY=!Hn(vgf#<`tYw3D2Y8_GdV=Q(pNu zApu8EG)_5Ac}0n;tPK3lvlj6LrVN-AZ4Kps_xo@+(Veo-#v_4}WGLwQBdU`(2$z8F zq*DISEu1zHqT$z;f31DUu%MunQ)AJ{&RH2?2eN3ogUM@CirYJ7XT}7L&`;MPvPxLLjtNR8#|dX>leE)fxrvr(!^5kGcsyjQ0s^}R)tfN2x1&M$$V41{_*QwDJJPzc}fbUTl zj3zB_;ek@hQpwt(V;hw?dU{Vl$RJBVGkI7wVWa`M*ma)OD_3<`q?4+Im%K43V3@Ki z8wMy*AQ*6w(CGeZ2;+eT`Wr?Q3Zj6l z1M*;4)o<)=Ci?#g6e<@80|*QA5} zJ&kE01&xqh@lWxRd#dZ~ZsvPbyppeZ5KHr;DSDlM=veN=y8t0+-C!Tc!4w9FKsS^o z07Et6C`CTKq~?7JXqq1s07?7fo=HO6gL>?_jEXg<@&}qAjt_!+2B51(=M)#~q9K0+ z)HAFmcF>*Hq{4c>7XKNqkkM72Dlj{+{e#)Z$MY`C0NnefQwNaHsL*{x&k6qe-e3Op zpTGZ&&!04ffSfdX@L8q{Og8$ZZ{mgv+(i^T2UDmx%C!`g#6=aJj+qQ|Z;dWAxOA8F z7==|-Ts!A6pErU1F8@}Vcp!kc;ZI(wrL=Sd{? zy@-lUlt45>l;hBE>TDT>XNFM|CpGH)rwRJonv!Fy?uXAxNA{z}?)`@p;tCj~DRDjF zm^V>Ce=f*1G!mP}lVj1Lp%ZL(EgNz(<ZQ{viM^(j<*(D1Ro&+#np6YZc2qVN#s&1X?)SICC0R z>Jz&g{O|SxGS&15Kk=W5;F-xO7;;=PHZ}qK&_rBDP3iC<@ zALdb0B4UAJh0E+&!(&sEwv%&kgzQmAV*V&kc(SAB8RqK zP-8IVg56Qqe3cxZ8dz%1x~WU^9vT<%KSl%qWt3ftgD@0@5s^}45D^h6()Dyj^DrxN z8C^*!We^c5BBe+To#f_AlXf0LH23e^b5C+mS$kMVy_8;^UZ?h;zVzPfHR!eI|4Q#q zy=R!I9Xw zRek?(dIQl!2a)w!XT8TkHG^ue*T(uAn{+hWbBVu_>q+-f@ZDJ7_599^i4kNu-xFQC zPA1=lU`Vl*$e~Wr=^lI`P z1O6P}Im#*F@@8WNE4_+Box0pyJ7Rb6mVtSqUPg_WxE_4u6`du-+^JbG(=5=~0uPDT zgVf$|GX`=wzAEyS;O{suxRH6F;|~7;pNyJ=*Mrys9n1LwFG+yy!F)!}cJw@fL6M^l zYPr{;hr{aQ&P?3VXHXyAN`F`8w)Hlt?=;Vykwrb>5DiLA_d2=RRsQ}8zM2}~7JX#F zOx>N&i7lAT4E9s|ddY;lj-2uP-4;&jU@klCMyq@8|C+}GPJ8eqK|jydzpRDtN4Oc^ zb?e>($NNj)BRO*VxWh*jXtuGMa`*M0mU%a1jo4M7(FWVGeZBa?rPJ=>`4Yb?b=-^; z{_!6)B>)}Q{XY{3;J|U0)HSEBJQeBjNMlrIbaqOf%G_#W4{U}M*=CfssYxXvb5!yW zzBKc&*=(Nnz(iLa6=B1M%~qY4nlbC-p}6Y%eSLrV{0r~bC*nIta3DlF0pE2?^1Lpq zfRqzpI4UD9h{icE6N)7Q3L5Ea^mO27*|=54(&ynrR ziM3CPa4Q@AxrGc$8ajdZ7|vd?+CO2KpP^n@Q>u17_k1ToHM(U%8Nc8GZ)t%N7=d## zz5n4@(rr;s8#$us1X}N;@VuEG(VP`(v^_^MO!R#Qf2xC3I_ExC18{&@@zD-gi>q}) z+TT%P*|L8#Y@TF}Gf#&z$}sz-`P4aZz%@!Jv`>;42qz&k;%aa%MxKRP4@zdW6R0~r8GWXLYjiwwIMP|n7Vpc zu=9XlSBe77#soD@Bl(VDeIt=U&y?)exO&cxRxEGQTQAmCr#BlR7>50MN2gB|1dXwc67kz_&YKmd)n6G$2YmQu?>hx-g-k7_f2Dd#<$Xi9>I~yY#fm z5k`*nX%V&8b?P*ip&rhDXs0kdPAH|U)~M0WFmpYEYtsN*-!OU@D>j%>J3OYR!5Pk!V}k|>@HqSBg0y9x3HtZ#Hu zM*CCY^pC;|=gT*RYj4|JWKBnWKhDphsEQyxh*;JGy-#O+P_GNz6hgyMt40B7Nl zK#Y&j`VuLQMC^Z4@?m*hhsN8b6Rkdt0!(4-+WGEDpCX_KjL=U-9^?6Cpxi6mGph1} zrT?$Yvq?B1uM(I2I{(FfmewZF+-suPIuNkA@CHB5-wHK zrrTiSpa*yeP*p#=c&NtbqtQOH6D{}7{(=$YA0XrWe_tOFq!L~f#l3PqjEoK*Wn9+Ob>S^O;G0M(qJ-8Ji3Tpe|oXiA**h$`eRI!q2c*O5=!(=gsF>z!Q# znMo6W0NSG#hLF&w7QgH6H{y8mWD4bW`@rc2O&ruTHnsqBy3>cx%LvQso$wS%Jei&? zq#>%e5&f-TasMk2M<78-4;Xjde3HiRawF6rJ7eTx9TGHD3~uy?k|bk6_2B)|xJp&-zLZPCO0_{o**CB|*JJIK)TsP=-w>$sITnC&H z8{RrBVoAFc^Va?_J8k^%=s>$6nX~_oa&vO*-$;6<|ItFr*Mah6DL{#?Y>umg(@mxAY3lDK#%xQ9TPR{pxtwFC_MY>8wNZ^gr_GoPU zUqwGw$DK{i3;?Fx z_jF*s?{zJ{$oNIyyS#L{0leM0YQc&M| zid&ck$LA0!z#md+x7%}AR3tYw63QMfd`G)po!{rugPv#swwu*G}{etlIX66z6#`a(bw%Rd4a%`@zr*#qUIi2f;_F<6Y*`H z;vzHqpGNsb+||hqH|Yxki{r$1aR99{>DJM^EVIvoFMc2fT*^avbjQ6*Fc2aB*kP^h zrkX5?vzY<7H?-h^cq{Yzb&i8C`TYbjdueyA1#MpBN3I^9pF)p3h8-bT#)s6P+yq4w zreTkZZBUvuLd#k_+bzN*V$vAmakmep5?*@q?7I_yLbc$v*N;3<8b^M~eh*?-n~g(_ z$U|4^|0fPpU%V1fo%MEZlW%0GD#%!5D}cGp%tn{L94qX_++v!z&x8H+;_&Ml%F$M9 zvfBgP?0|fur7*LO?=)K+TK*Gh-51*Ljj&D%d3#>=Ndx>d`cyxiIX zPt6i2tN-9t%gY~IdP+zlf0&I-`cV8c%i4|u>(txJc5lM)+p;busu6bgCPN=0k3I$& z^izJ`yR`dqQE}Dy^>b%@ce_XU4S*+^+HHsKXa4a-c`WznziiZ^vHu(-TfIRBe&1p} z%zJ|)Wen4&l|3v?Q$jSqGf$oUkYZU=_nu?Egl$*Q2MYeS2|AKI>}Uj>`69igsX zcw7rwpb;Nt9&Z8qpCf|D>T?#i3Vd2-Dgt69X62n|mlG*3@3k8(%vN7Vxj=IBRpnM<4xnsy%$X z`=RofI6Z@-#@KxwInura5;z8MT`m(hBC76GWHg|z5jMuy9iQmCZ?*Te$b@(`1acxu zTsIcY?7e8LfvYb?KpVvM)!!l$eoIp=M`h;;w=m-ml5N9QSB6VoimF=bcJN8pIweu% zEyz!;f%6G>0~o$&XfHK{OQt##GO#h~Z>7tu)@d3Ub*UV!BDPasO2}y(d=q!K(Yux+ zVTyJeFZlx=%gZOc9#lJC7tDn82aGW?D7{R<;GB5?mnTnQpn)t+ovtmj@yd8#uBTaN z@ka1cpwv;GV{*k+la=c}ln_E0Q9UVk5WuXAyJ_65iOa9RDyGG}t}K;a9_rbkTxyQP z5TEkMd%~OI|alh)k=;rS7(^;K6S=wAsFcP?=>>gSAv$k#_ zDHo$BBL-I&r-Wa3C0O7g>xcbbj$IRytw>ELN%v$?{M1aondK$_pq)qL&t72woGd3^ zC%T_3w>?4?c1~9%bQY@447=SQ+gj(~uG@S2Kctp8WoyrJi!nd~Uat?Mm5iRM{@Ohw zzYwxVy`b>Yv^mz2AcVASSaH_)j!w;{1bbFrAqxASg!p*CG%T&v{);-PY6) z0*+osWpl{=^A!Zr;gOOL$TwoqXWn!w@c?4~ArDjb*I#DtrUJ!^Z;VCg#IzmR^ooiB zx4cBbzkqe~F9btvWZ2dsr@sOZH?psk2Zj|CSOxScWW@A%#(KiOZd*{N!*=J6x{7W^ zuWHe3+`{}<&?#)JrdPXc(&_NSL0h~ErQPsUWG>P;HTeg)+F`c6P_)#LlliF$vs_mhl&W`*0gX%T1iP@y3YAG|kxQY_YU;$>g()#*0sd_r&}V17byU-DWw z5U;Dq_fDDEuvT(4a|JeG-~!|S*xJvL~R$;km|SkeR%n1$-x?p z#i1#(i@i4fc4sZSXKo&MO&TnRn2k-eT5p}t3tMedhd#&LYsqgA_$ToW;*Q;%)QUwW zpDv)JDe%I+F$a;T!{Clnos!eEikWIneo_S;nj_oYPaSM`aMgj*R*BICOI8=z4%O~G zxW3%5Zqc$<9Cv&^`V#Q_NZ|+Tyt;uw!`)z<1$Lj_Q`Z<5ASjD~NVVDbJfnm#(|KE@ z6cH|XZL^Qh|JU{0DV6z*_@h;*L{X4r8g92oR`{lf)*XjWtWK}67*EQ5NUU`<<35Sx z9@n0h+Lp5Fza-zl0I)jC8h)mAdHq zYF9E_i|>&)`s2x}po~%PUbA=e=ScJ%nYoiqepls;&fy-=@qFR%%3JaDL(&Pfc?le< zQsUSS`v?{hj1U#0UpU?p7d;bvYVCdmS_pQlt?MJR5xiGOtUw{HHQTFya5;~KUZ@d% zW-ndT-+(flqU!U!wsRe9#cRH#QT66L&yxCl7jE4NI#TO&#{>4%^C^%=PTQl{@Tb+@ z=DH7@jLM3c$ZyRw#pAE@N$hy6WpwO&^nrG&$&8vj`|wSQmKx2p2zB?0@P0I`YF^yk zTNyeBiC+D#$ktO#Cq+*e_w~fy27|1Lgl8_vH10yg$;KwQet$J6iHD zk&*)-+wiosQ@f{-4 zvJIxetM}mNP{j;kF@bR7l06%X?)8v1)LnHh6Rtg%?;8@ntIdDQwM6Tq$G+WJy2|y* zOPcVz<~dTOA2nw=y_$XCM~sv~;%%L44IYH)6yk3q>u!yk*zwote!CsX2+PjCYTm9` z=EhN6Cd1qwc$GI)g1E*Q-cR!0eA+^z=FrTqh%}v=`<^x_a)cQieZBWjd#@B9ZBzKT zW<=4~QNArNR(2HcBD)9h(YW^oOJ_SoE5HM1a03|et?)+;=|7Vl5?41zSJlwkyBz8{ z=4HePWWlse6Cd1~sE*{4HK8i7egF)9b&njh&%_Rv}2m3vn=j2 zMSCksNX+g-AVoNLMPZ{ZdCTlJX||2w#VWonpor~XJ?0L^V(TY&^(~*Q0qtnaL9AuV zSMNZo1c6f0i4ey0By_;WQDhe0@QA#+mAEg*s-iaCCuuLDvm1V79`k$aKuzOaswz+o zourg?E4E!Menfw%?Cw~?Jf4npo=W5%R08vgc1S=&1Vf@-3>J{#aiSSNpg)7MGa06L zEZ%9R-XU$GhI~S86O8`aleNAXL$>$R9?M_!?sS|Xe?jPL!v4uwL9uLtxjSN|32?zM zHMh;8jJGKw1gt>IOVv#S3yIUgdaJQmRq4q#Jdpe@VD|BF;VX*kC>||%5o3M@ld>kl z)RN&16Qs2w)T&U@-PHA72g=CH3XS5?cJc8WJbWl?WZC}$N&o<>|6l#TmVx#E|L;Kj z8;Jh`@jtfz|LcMDejvUN#P4y*=lur&3>E-gns16KSrWxZib8A=DMTC*sde-cdL6Te zeoGjbjSP1g@9S)$kV2#=L`0;peUhV*=XWBiKO|XZ-n=(%{5>ab{pi{szYh0nd%te? z>!iOo_v_mJxIN$R$NPPHzwh+?vw!#Na=*HM%x!+Zp87Gj;r&|b_hamr#ti#sx0#+T z`ZaGO&3o?uevFH??%&=fdj2BWx_+&Y`P%<}%=P#kbFZ~#S7f?>|La5h+G>r5e5x=mww%c9^-p_;;O%|nq#HC zd`YHF+2cyzKc}6n*K_f$nZ|5-J;YpF$$8X#Z?dgqddL#r+_W#*54*=V8_jVj{;<=t zZ7+9Rv)*2@?zzOkO?I0}r>8P^CtWR5m!ZeG{Aw(JzV`ME9bGgQ8HbW>tabPD@1yi0 z8^peTmtUMEKf0Uh`;FuzCguAPGsNLo_nG$nB7I!6*R|$cNViwn<30BlI@)X9rN(W` z8u%jmK5LI7>0zyX5jQU-j)~W?^obwrw8oM!x(>-zwPB)=eoc2 z`w!W)SMN=^G1mPloo;j9h^<|7ehe80+RL!mj$A#_-$z}Y{AApZ4I57Me31LH>`yK8 z`lUHn#oEi}`bNFRmv56l|0QYn{fev}BS-pDS?62!dMP{Xau?wVH)%}SCp1WOsG^cF zW8Ac5WmyzWo+Q&*-ZaT*2`zLQr}bm!MXQ$kROEh^_MeJ}jlO$YXlJAdnKf5=d+1ZK zfSyquqxvm0mt>4NmYVOi8FAF8jQ6q6e?DIP`5%dglXVKTP0M7WdEr|r@nHT2-NDQ<#!O*o#Qp2LcL(9Z{>BooDo>~^?PS0?_U8>Q63OIO|%%h_E zkCwrBE6={PCnXkCi7Q6206<}q&tj6D#wwx2nuNho&qz?6bhBv&YfhE}tvl72v!2b% z7!tY)n`IjP&2_tym;hKGEfd3&WV}d*#lvNufBcpa70ZD8z^vr&&z5+6DKVA*E#kJA zkUFtznWu4`snq^fp4FT+mzobPbGvFx{Fb{##0VpXswSG7`tjBh$5U%%&d3OGh?`tT zm?3^2r0S8}ZI!NwD;X;*?}l2(?OQej4}^Y>lAGLmDIs!58-$J*EpzuU?GbAXTJoOk z-BoNHLAM}aGcz+YGc(4_%*>7{cFfGo96M&_m}82WnPZM2W@gXLe^dYxC1w*J-AE!FPI*a;$|7=b{-otN)it}{PW1V5FbI@KUBt7T3m9v^{{4gcBLg(}CQpbOx3N-WF6x5*y|iQBU~RnpZ(0xR%=ehV z$305re9IHsHJG!iMf`0Gkk$W2p{hZ@~4KbG{_X_xK|E%t35FMB^;d$&LO-(k-nU`n9yaR^#2zZlM0mqhah z?}^{!mlTmCESR%{x~+CGAl+Qay<^0)!yVN*S;9vKqOb2zY46X}p zNUh>Eyn$_m?B*}qulXvNUSU%H#xift)=EwmbS65d+gR7|Ol`%6mYsedC4_BVB`dYi zxhg9VKS_3mb(x57cY)HE!|NlAP$XxfZi7Dr4f&xxsXn%7 z{bVFcu9(p(J|VpAAaC6F1*p|$hNfSizkXQcV&9A#RT+kp2-0@2a7qOriqjQ3LRF|w**Vz(^y8DNeg%)ks9;=5ME;A!{lP}CQWgu@?{fDE=l+%_)Sv)5-xK58 zYTyg#s5w_r)Bf@^A09P9oj9O5LZuBU0HUp!-_TX zW1-TDl!(=P?0wGUkK=$E0&X7HErktl6@!4*{g+beo1Lt$kCoJaBHEHWjzdFEMusvY za_U{sR8Ol-`1n_pi|Y`6rU^e!Ls+}G_KSi? z*jD5T?)S%m)IDYB^YNarq%6*F{mlsVD7{jFdwl_{&Y!riJB(?4gTd_bXRF^QFD-a07h-DmG0;BNX9JCy3 z6yvu}vs3wT-%XnseqQTw-UP|bhH$FD{!k5%LjMDMgtdod4aT{r2qzd|#ln_D5wT^U zO*QQh7Q-+e9RHLD{-NMlWn+35b}7VN;P9M>#`Cb=q>mYz+Wp!w9U?hxH*U+Laf`&v zeZOh9-BDkkBau7(lVG|!f;stX=8LOI@WpKzb&;RLb~2Rr7sES9b5SmxUDM1wt-zKu z(rD;hBP4@m+1WRmcdEB&cWC3$`f z3%R5fO;4god_xeZ3+=f2E?>fQ3Vz_`L#^v9>OPK<5qSp9+}!L6J~b1Ir*VLt-x zy<4e2q=_P?D{frW7#1r#ibn(Sf~NUS?z8iiyDf$j3sP~C#WqOuo8e*@jI&`1a$i~c{lMTlW zSvHjsIN=>wEs3k!se-9<%k#6)Ay)&Hj|)8p8TXJ2ygkg`FkwS=IP1M1T3sFxk`rzu zrIImBP&^)r?9}nMy-FeQxXQnG?4{NZN)yxmy?vqz`ZijZ$PPM!dPmdZr=)wse}HE2 zdGFyZzIz9On?r#qrTPC`^G6#(|98++IMWZg-@*Ny-Kx}C6;`A#O`|A{!a~>Cw7kis z2@H!K^XvA$-8*^m5+%+Yu9>fS^LQhSkB|SE#hvwqr`jU0zm5BKXmxOPYn51}d7@G3 z3H!pYGX5uiZnjqT``_gz_J^S!vgH?%&5i@&E5Xoe_s0k2zr9gPeL5P&kmXY#s0paL zD3RPtQe}pwv#SI#Z?6icHXi*d#t`9j>z>=f)Hp*-#bgy#8;`H|pDefc+`-Pp-r$Py zf18ItukNBLTFmP0(lFGF3~GUMZpzj3-)oE%J*6vtAiRFI?~68VTO2imEa*Su&U-)i(^hus4p7gWr2KxC~LVe|6GYJa>$EQAvdt)kCY6w zSQROhf=Q!8sfdX?F4fTb>sJIJL|ZMOVGBy(h%DU(Y?<_c26+E8+n1hj9fucnx0(gl zKe^wNOI(2MMBW=I+azs;%HLg)36kz8_8sEu2beW6fM6WjK$G!yLV3C9~o1kG7WJsVKG1UgpdOR@Pdh^R_K}{9y)zKu)MK z0mU0<%d(p6`3w;2BZInQ7t#VQubUjMZ3ER^J107X>b{~PM=X-TP;8HWg|S#{8NPh~ z3QT9|>d;~k3}hM=W_{&%f$%Kv{fecjY4`^Tzvuk1?(<-wh2@EJiBQEO=gn_<4_-u% zg{Cb5$aKA5mtsX|U9=R=>jXZ5hE77__$vr_|BA3lxCfWR(agpi zmO_e!h3xtv1YOcHU9507dtY~#F#QNs_e$kku_(P+_kvG_*HGTItBpA%x+Rjbj``Kz zm-lZg%(3$~4mh)yu7(hN;c`sVc6WKqDVNq2#KxPyDF>NxISi6^gb0W+Dr{rrS)o## zjs5JP9rpC_$<~}G!YUDPgt!A|xwXLJguW(UqeUKV1}V3~w=TqJLY;wVoVETzTFj2Hjy zykz7{7Ue|m{R09@(qcQ3-`)cL!>UbEB2z5HI7VGZC~=L>gP2UpjMM}PrkodruKTIR z5QTAaiFQqMzl}p|N)k7iUWw!Dn$KmD=rSV%g%e7heI25yJafl2UanDj6UV!Kr}pIu z!(3;od*yyWWCxh2ht>G%>-EB)fR6X>Pl<>p7O~?gO6{!wGXE$A${@>lv}tiSlp_WD zZj=^s0Zm&XvUH>wKki;qP^$I)8mDozw|6Ps=`%5R+WtnAXa@<&hA1&VH8<}gHflK~ zP#S>B5C2z?fs3z5IL z0jquxW6HRGJ@I}NUFhL`4OACdIr!$fUs&ExsNY}@3TF3nmN z)jK9lfh$1s0`CFoRvp}@Qq`OSLERn2jHCCT>`Tn;6R$+&b)kIS zMfx0@i;jkkRF=bsPNA&A2k;dXhW8tM-rdgeq}V;nNvDp&oUW1{9SUqSJ^q7-+)v8c zq0zAv?Su$15WM-f@ok;X4He6Cca`puOT)#=cCSs}7}4u$)$+9w*!RzY*E;!!W>KM);#Py@ z>I`sSTfmSf54fH{ef~b5+VH#R^IZLn9(5A6G-wY(!iqf3S()(5lunjh`}6LaAD1Cl zN5Ln5tnNgPi5oo(eNU{DwB3k}7oz>vcvp{!uYYGXS_Y8q~GuQEl>-V zYJ2fJylw;wdv)bjmc^!5szmr+RN9}>tq1ugj$*&Ak9G^o{@=_K(Eo{5S_q(u^;GDpoy^BXEl^KJ4_ei1NMe#)S*t)yT&hTODP3dRY0GV@*? z{*0Yi#|W+HNZIwia!EOyo%g|5Q_;JI%9t6uV>w<}^>5Rd*`bJuUH|;y($|- z+FO?q6lTef97~i7?R%y;zLMbg;Y4~T;b*d4o!i}8BJUeEr`<`DY!ou~9_euzV~xJl zM&>bG=9#OtwW;a!(qvmfxiZTBNoE{eQVpL;6`|;4S?=WRGp4&+LsvB$nwdaO?bLxG zWg4KkC=y%Klq;ezM0>}a=AADfqduOUn{bza_S`fgN4SI}8;@Fs$m>^=lAB}p>j3qy z_6hEXFgoLB8Wm$~88~bDZq}JKDtdDnq+@W2Jlu<#G4y5-%a-}hzEEoo6nmB~MXA15 zL}7ak+;y?u`%+n)$g-!Cq~$)Mw>|AOVdIAQdN3lbeF<=jLRwx!>X8m)tO3!c9R{C4FV zm-A(FTo(Q1)#6e%QXTU-xAD2TyYQ~53O8DICVYmYyzQaZM!9RyVS=TC`JnER@WN$( zzT0R%c)2;R`iq)H#Bz5B+z+>in%@=ZaCP1RUc+U)OIqNGHc7pHHr|sRH0K&aL+JrI z*;imlVJfxdF?f0CV+<5;y@>mSDhMi9zY#i^J31r6`iwkH5^E)d{ZpJGH7mITbWZNA zp-$Zke{=px{L>(6yK(DNqz*zPk2Ra}bL^|K>OKi2Vty2yEqdxYW!dt!O95S$oJtsv z+pxZ0)8&8$UF<+BP7v7vEYS)OGn^}OOt?GtuIML-YlaPh9)$A?M>yoR@(|7{aX%O{2q$Njn~wpjr_gEDvycTd%Rqt z8%LcVg;kUJNl9d~&mUjpb*aRTgw32zL?dd`J*(g8W^QCS_FpNeCeeuehBrfX=S(*ImDmU(%Ip(Jf$JsV>cE?E zmZ$u^S~p^k$_Vd%i8(2T#(k#g+A|Yc?Cn zbL4u*^O`t>zF284o!9Cy7o__|SXcWg?o?vj#<#5(X)>n7S*he}&|5!hgIDzN%evsp zvW^~jFe0EHuxgo+!j#D;qUhl3M@QI)qS=40l770l4{URLteN^)5u58QQW9T~FX5>t zqviJum|^7YUP;RP`Dv#Xxn86%9ebHNj)cZhs|=-sXqKAD?je=9;$>$GNGVNt;rc;Tjm~6s^=E#BA%R|TD4bhj|*7vV@x{z&O^)MwKdKa6> z+>VzPPx)#0nwP50sjzY}{T`P-?R&D1F8jOj7W?ep9Ifcx(Xd&N!Y+?8hT zi2Kj4Rk>fnzM}aq?-C%*_jgqY=f#*x5{KRM3c}}TO(qvnNC#*AWiiEHwsjdK`MOm$ ztF`BbcM5YP5+iRBA+RfuipU=!5kpN)pQecO$j!p8iPl;L)Qt}NE{6D1(N z!sN8X*su!*MIf`{1ObWEW34OC?Hgj&ybe?W$z9^@TM(2$X!&Fb_wO3CZxEravq?vV z*Zwt>XUS5o8)2ta?3uVWiJT)&`YNz*1cjxqg|kVR{orof+i)cB-6LCJ3nf*Sm#*IH zrq^4WViHK+!^5UXq?%M6q(_DKm31pIhp*M~-jPX##ikZ4T>c^0VO>UeZ#-(ZE1mcA ze}Cj5#C`qhJYw;hsfe84#$0pg4>7<|Sd7?EWHbarnh`y{dms1S`q|l-@P5vj9QNhT z+`-L>^@IuA8}$n3#r|TRW3uuR#GtbRpM3b{{&n(IZa%eSU77b(%Q5TkNX=bYiFJm* z|1{oW3u(Bv+O}8ma-&ysaA0*Y|LONyq1aLdyx(t|V;uK}UalM$68&W!%Vz%XVHn;) z#nYcR{p|yNe>)j}(wR=THGOiYtqN2jWTnZ~P$ z8}6WTyB4-+L+zojYqvxeRJABM_sB(FMf+FL_h|+wnmWGGU;H8N&P(SyuK>X3w7rShD_sD zb8xt?S}~KgX$p)cnV_yV*0Dc;*d;hhY$95RP~nM`xj8oAUlF-35#w!+A6H57`~z#C z7jeS-PPFWnCdQR9>ZR|Cd)xWySgbr}f&WPicsH5odjJIrpMpTI!A~M+y|8=#Cq=;U z<{sq!0SSf<0p<@1g685)h5W$G4O_oqrxKxEm`YpqM|$N>X^Y_K5sR!F?0+)Z!c+6B zMvzoE>?!Cie8-t`BesS;86xbl>mdgJ?Ia>7f1OMzPGk4M-hiMvc@Sv(%}BwOOK8B> zy$yzr;wx$WT6$7j(V>*fFpFzC4rbP%4(42(pT1dq^})siX3z-PX1UlFkuO1(e4akG z(1g~}y80W(Do#|+LFL}=xZk#AHW7J9H;Magx_C2u`3Xj6nB2OxO~y&egy2cab3w$2 zf1$WvM>e5x)tyMfH_`l;!Ylpu%Eyp5w_i91=p`H!7tkkTam>j{ynTNq4Hv0?Nph*O z`0n*N#8Op11cDqvpgzTaWL8iAj{*ua7GPk=5MV(*Q!LN}xeYL>Dr|Ln)~$!g`HK;4 zXec2P#!t*>{zOCvnTV(5VNNEdD)L>uuMb`k!hMtB@SKk{kzf`yFOTzR`{W&gS_T4E z)t_>0sPHWbvaK;sw>0&e67bp0XIrA{C<@H>Rnte58ue1;snA8(*$rgW|GE{16RMi5 zqAooLt-;?%)IIkotmwr?f5%*oh(Rr-iLWe>8C31hutnm}z$MWgg;pMr2S;H_D~+02 zc@nG^h?|DH3pW=Xo4LI5nERGqrnE~6wqB*fc-ygPSn012HtBamO48#Y%8sAfhfI<( zK-8;mqZUNM+BqP;>ScsBGGmO8!zeERJ;MqD`GP<}72se-@r}w5YG~44+ZVoqf?e=o zXl}*f(CoIes0~f`Z543A5eYDYc8*Tx&xpN2SBOZD@zIT0d1D>CoH-g7VG(yS9#-FachmtgHBH`Ck!+!@Z%gFD$Dx#n5XhC()&@%|BYx4D~FS6Lb zS)#wo44TO&zsU?d#22~FnQ}H#uQq97S30f{%X<#aAmDUR_7eA>P2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve z0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00Mvj zAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe z00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r z03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x> z2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mA zfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@ z00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve z0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00Mvj zAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe z00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r z03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00{j5D}V|H zh5(HL2^U4n1O~Q`ab#{21V&DhK7li;6^wN>_OumESwysz2+jZQ z?LRo)38YQ(>V`rvFvB!$hkaj<6|1W9iF?-%^f z&n9kQH2-P;vu7FvL9YC7H2O@2ilR{e^D1f6f3Bjo|EFJz7Z6C~6a>0F1?hn5txUsj zf}RA?Krnl~VsDA(AdnB)Ptr`kfk3bvEHI^cX-^152r;$I?=D{D7+jwaHs(zvI4`uQ zKPBy4IEO=fors>@X1JJ`hCDfvh*kZfOj0nD*(XX_UU>-cET%d?tg)*@FsH@7*3v#bTI0!oFi+0{)I+8%XfvC`96 z*;c%KkF`*WV2tHOCmKV3ZA-)CB7yQtfTUcavc^br9<#W|l%;(>#N58tjppoSsTZ zuo*ERkQld#A@nu}xrB7ny=ciHLA7E2`9h6^4>LY{)-W*`SXxT$`H~01qxdXF&ST|m z$POjXQF#-sZ`mZ%^F>B?Q|sq9i;~JptJ_$&te~@jh378;m<54?$@+C%5hc!(+0leG z5Gp==>J{Tre_Y@FzPXzjzZrGH4-23%sVq*<0 z9*_1L!BcwF(1d`RK8U`4^yR$5^nKI(_*>1)Fw@<#^=&CS0$fyEIwbS8qhY5zn2<(Gi!(`=EC&Gr zr}Qa07`f?G{wvj##+Eh;m)T0UuxdAdq-!N`=?1+Z$}kcgQ+1!Y@YUMK;3w@pRQQJ0 z4Gu)E6a#-}PV#fU?UQg7od2%Rztx?dB!Yp}BcV6ASbRl_gUEk1buimwsKN9Wr1V*Q zm40}Xw*FjoI2C-BDZH_2UN_v^rSs|6U?h=8F1d`l24<5Lqbb-^KdNi*{2g?)p-Jf% zB~o8o$4EmX%N5#gAyuxNUb^j6CrD~dJg4l)=ZQ6G^x#v$_3Qw0mDU)Y0rIi{-m8Zz3xQ z(a5f+Y>Sr6XzoPbtAqwG-*-4BI_FnMPb%pSQIpA@=clMc^6&0OywPoTFGhkR=OuXS z6|(!J2U`sl{Waarb7U1r4d<;bn~(H1wa7f_52>CAUu(yq1&T+?#kCqpz|FWKI#rvI zQsoa(!uphbcnw4*UEsUWUSeJi&Ss_~HeljZA6C(`AxFkM^QVl$e`Tm`|Gm5jdW%ZJ%Nxwm_m~)^GW0rj z1IM~`a?s|gLDM&mNcQrdlq<9*Onv@J#6J`ZFwcXJm?yK|Hlal!iI;(zSjBNE*?R62=ALAIHq2fJeN#0`?+kC3U)9bw0FQoubGqPDHWZT zd>O9|aU`;3OnV#u05;3lwad8S^^nx5M??AJW4x5AO;wCA@;^P)Mrg-xmlE0y77mq& z6tA(Qxl0Y)0S0CB^+$V#7p$^{so&NexY`=6t8O(mO4t_`d*DJd-hZJ~T`&gFA|PY- z9l`(7KO<9Ddm;%%L*z&$swj)&E=Z?h^YNqm`Ci;EaXJfOmeM;{NfY0UAHx=E8_hyhnPaiUUs!S> zF;58`?}m_>W~9H#q>tP;KCfEW7_v-6 znl&aJTzwa{xt?Pfd_@)jL1j;8%PmRPf%TDIk{luB;qKxuyMlP3{%7B=^ z)5HCO-}-r-Z0D*2-9$(K!zYj%d)3yH@i1wy?k`RrwStOB%(6bN%WEN2`+_0A#+XF< zlGJDow3o1$#FIZl<<-l@?AFugRV^Q|_Xg{fIqDN#K;3jBaQ%hVmZ)cMBh8j7vAKO3~T}?&IiSHR< z!YB`PV%Xa9TDNTATYqn8}a5yUu=a2rQVp*;v{^!_FAJ$8DcUN-;qC`)_M z{!H_LX`wTzR)4-#n=s$PYblV7u^fzS4rLRn?yVQqk_z=QSV@!$)^*e5SS}4cND9jI zVzUg)byUE*pNAV&tX7u`LLY2GOp)PKGJjV?SBV8XuA^cVQfN;%k))+#!+H4i_Vn(X zM(Zr)ueRj3Eq zwm%T)Yq+te4+=hAcgA0uJy0;tSib}+gMn3pX6}14g75#meTuyiMu8x%MO#6j)8M-R zvL4jIOV_a#P~tl(*bFI{q1WTe#%dVEFbqdL$m_xftd^`&$S7#9{F<|!l{J(&L19Nh zIQBxtS`zVGZu5trgAn(@V=6-4#E+v~!tnNa3NLvk_|KZ7#E2jLw7H*YEy8mI)+)-n zcsFB0cUM~29aIiwrfj)%G)dBlQe^(Z-_-tXsB>p!DBA;1Tq2iE`q1@P6?s}DijNa1 zx7a(Lu&?_JsgNy##K3N6H*?3X_VBwNc0~_Y7{<(^`mnmmouK1s)U&h?jdEX4hq2wE zM6WAIP9e1?XitbC&TR6_2*1rAlr#?TUouCcYo?A=YJ&qX{h+;DaIpV%HUbfK)2h4H z4QISlLg-1=TUGPuKdVK9(xGyo;Yq(P>a@J9wZCR|JnqBRl^luP4{A>td%aec%^X(j%j z%rr92d}90(+Ll`J->_dqB2n<>sWz&}?02``-hV}B@%0!fvC5+>bVS^%EIpMeZ*(em z_|Kbu{MjO$zEYH+6SJk3LJ_jv!{$BNz0F^<9y4Vr|Lld0MBvRx*+Lf8_{bbna*zIO zR~~{4+36l?r`bW95`IWyE?}|SWDbJ`;iDJ$(2*o;@2*x;IAQc@zkEeS`qdf%Gj(z6 zgrnEvqH!jyZio*qNQx(b)P8AiV7g-CkBYsa&7b)U4k162hKWY-<7l0pkh=`&pG>uv z<9i0!6&R@tgCHCJirW1^K}tf$(vQ`}MOqJ$?R61e&4JBPqwvJsjolqV(GP@fucsXe zJveBXJy1`fmx^koa2cEmL}n4&DuO0DRRwIu48z-o$ixtQ1d}CfHL6v$3L62TCk4`! zZw6bxjq11AVzbN6TBoyNm6@MPOp25PYd-ij*TQ&(m#7*awGdG1t}K*UpM3+tnv?HE zNvy@vX?>;7u{4VV`JB+izY~D$c7jmrbU=ZPv6UH~hfZXakNyzB^h($fO-kPg{zlCi zY252zw+V&dtG9;vQ!(1t?KW_z`Gg7Gw?Ar-X)S3NJ-LLItl!~c30gV0fjX}BYniLY zYQ*?Dp(zuhxC7~OT&0Qw@QgN1-NIhvkIq-6`HU|*C1SH>bQfEV7ZX|Chs0w< zFgE=vB#GI+ay#+UFBvP1%kkZ`V^yl<>>qW#$gIv7>+d>9dV%X_J#yXG)oXiJVR(|@ z3gI+IUY%w6A5VNoqvMQ*)B2=rSNy1}@Z?E~9_IXuB~vUT8y!{-BwaiIiPUUXfyWCR z+%M#NwUErgXIbWbg?n+_M<;jQRwGONCj|N{-r}=Zy%(&ur)v5OU=Bpy7 z>ml0aRmLy5LH3Z$Up1wR!2ZPK8#ITUjuw3JGfQdAMlJyd5cjlBtfO;{Q_d}L?O z$;BXkuK`EPT6e;Ds(!j*vKi2246j#vvoMg7p@5H)XdZ+Sawv0qs$)1Er9~;M@>7-7 z2fn5j3QJ+5RlkSiJZ4jL2m`Y#YP{t4XjNl|r&zm5`85i>ChUxUGUx}f?Q*m6GJ)r( zhrwQfZfAVc{)=y7P`Ev75p1k7`M-&6a60*dO`#y0yD8GC$r$?RvNk%s%)bf9#4d_n zLGDmafBm5EpPTAI)h-j0q&=_imSF?Ww1hu_kR(tFI!}Z)3SvZxXi%JjV`Pp6SIPXNK&P^f=Hy88@2C9s@!OlW)^!CP zY11l9$wp09XB2l_&xsVA5ig!F%8ALygP!zym*wT;8Zf=j>B?)Yy+=WIPFgqQZ*g5` zyxTF?Q#o~_e;mVpDlXK4>i#}6^>j{OL5woeOO$=xfV|^usjVRE(fBo6Sq?T&g+}td z88`KXWVW|Yxdh7GJnP-j1y*0+vCGSc=vb{x^A-L5@P|19OdY20bgfE2;;}cHaYT>x z_M#Z6aXnQ1Pgm|F4BA22%d=>agJAS2V?z%mQCYPiM#pGuy6{QI9CPGw$YK}%DVqa3e1=DFkq`Qr^E3u+4NY+-A$t5?gSWX( zF3WFC*dESy1k&sxAXQBo$>$4n><#Dg+?7~8!jI?PU67$INOg3khzK@_D*+P+s|0UN zH*+DMF|XjI4PhoM)@TtZMFrcNQ$8R}Ll6KJdK4i+a(*rydQx!T-~2+jqY_fHbe?#x z==88J2&X?~vf>aEoJwh)rp+{D2%jQ=h{Re)SKhN1DMJ?GnuXF^n?0C+a8B{N=w?;H z2()t)x83l#ttW^ORZ!^|GdJF@{nK~VGk+E%;5mseZq8eoKcC8LVzag0p(hr=bI+^+5et3tCC6OiM?L@19pORmCaST=k3a zNxXDqg0k$Hm{p=IS}E{Se*UW#tW;`>n^@-D>zNKY=j6;@t`Z^UD~YIU+@RM6?(BFv z&a4b?#oulq}bm7e$AHSzvx^&=oKZS$6kGViGzeLlp6GFO#T-g+*J)6YD zV0>Yi7xd#Z-PhxNKRvb2&<<`3Vm_ixuw-)9L5h9Si_?CxyTE;#kgd?V?- z8kXhB?6(#IMzZ%4YV^4Q3}Qo2Qt1x&M0)4r>MiDt=6#M`7r0hjif4^mttRR?VZ3=x zGnLC3kcafD>#k9~Z;S?Y8*bSNq(8c;8`BH6%+9t*8J!ZS9xQkY}WaATEd5C0JLngEf!$S@~JH%QP0+SZ+`(cs7jw z#0&=N%?XsxHcKcgldFU50lSahtih6vn#q}F_HUAJe5x=-*7H{l;p?+2rM@#Zk{GK4 zKi$TU#EtDB?I$Ag8r_~9lJ}Bi9yaT+velWuG{3i5>3oYXu}x#YinlB-_kZt-XRZ1Z z6xR80cB*B+#&z{Szixv+O<|dpu=Aa`#0Vj2?7l`;R~?vBWmU%o$JEk-t61kW>ci=_ zsyvFXAEiTwn=J%mU+f8Jl)%^=LBVY1hTH_7%5AEr*HaRm*dJ^`E$)+RwD4(u= z&r^xLyPwXAJeUVrrr#|zWs=70Z)C;#aAuI6Cxy`8Ve?~T%U{>1LRZR?4!j{QbX*08 zDSpi~vm2L$v>los3;?VkSdc}#1zi4Njb+KcTD#KgAta)FU%YTx(b zax}4VBb(jJ>cHsqkG*ihNd( zsqv>xl6ItbEm`!lL;P{ZODUzniVKFSN$$Al-_pSah=_=4K7l2zn`WkXK_bDqXA>=E zJf)Z)WoHz0w*1m-NGQrfqEj~a7l<~q?AZk;)DsUxdBnW$oO9ApLpCnTA6k{LXQ$T?OF{)# z^HRSu*o5-5mU|C5xd&@}QKB;1PSM{Ng$mKz1#7p)oZOV3g2j4^{a+Xb4W;o`dozpE zN*(g+<+o1n=Mom06V7}kI3Z%JDX)XU9}|6+U)dIj*-Qq;p-GvY2HfM&w7}{QR}$=q zAwO$VL${~B-3AXDO^(u)IS#pxTS>oSKd|_;>SAA>Pnw^hOYzvk2cfIq!;N}l~ z>k*$Jb!x)dek#H8B2`X{NiwK#mJS*HYNN4lL&puRd^P>;Cw`v)?v_=ToOw!De}a5J zQ#?AGe6T@eYuF#&=*i7#-l2+a-e1w^a~L5C?H}gTF$U0gmPHL3sfrhW8h(PeVuWza z#{`W2jl^8zTioH{7(EyGcGg_9syimVjdWrzT9|7`Ip_vMXKJ>y$1A9WX<3IQkrv!_ z`_Pa_PFRN?W!8g8Oky}3z`kB1=Cc?!%OLrhk^32^(h_k-G8!1_`Do&L=U>_3}K^!;0t zG9mjIc>o0pCvBk)J_bJtqXok5ZMPr%XO$|^1`KRY6s*syXYXnW3d{?HO%KH&uq1;8 zADsa!oJqn)gtH#t&#MZn^H(Ml;=+3x9XEy?jO|H-TB6SeQTXu(@?JLt+f~~VL<2&y z2@56zLyQoEqKjIUt7YN?agr31>ip1x;je5eab_kfb<1&crkv$j*;LLWZ2W9jF7$Z= zdqtd5ooB}=o{{AwF38vw8eU^tYd-BZ%g~`&yvE>kt9#ZD55xygB1(5Ap<0Prz6SXn zCvOHtwK4P};m)!&o{D>t&wRg6eGrVmCCXxTb(Ev;ct1~u^GnOE+gtr0L1X*2$p7;K z4oqHVRCybd(qn=^QdIovf(QJR7tfD1lRe9H(y5TK@QZxgml<8v2^pPb;7wt zVMnwhdbcq>WWlAl#Se&S-WC60Z~zg3XiW*Nb}TPD!)w z#h{cxNQgmvXsB@Q%&Sw~@QmKCFKVJUb3%!-z0C5}Yfq%aua~ZX*;BS@u^E#?@Dc4A zLC5W@!88UhVW1JCp056H$aiJ6+DjG}&0a7tVp0-)J9o8%AI83qcGCnyN-?yGkeW#I z=3CG+77MTeS8Pbq#{1#~#H=u+vN@kL*fk_-^FmCjB1!d{vWxMgt_-R7xpK)=bHpth zC;HUtSc+&zO9KPBm@hb{1Pf&EJSnZDr;EknpEtkwJ`_nSmVHsXV6&vdw%}r#Z1?gx zn^^SbQ7^}YmUQvGk`n zJ>Le%E$ez~-ay_%hT~6YHNJdtFv#C0%^W7PeU{hf&6^yOvaOW0Y+e7vIg(E6$1yw?t_qTJg zl}BWy-ZB`3KQqEH%C_+uxE$Fwob8y6o#K)nZpSxyy}sQywp7GbQKWiI<$b17;>Q~n zlQ#-7mfZbQS$UjmRwR@R`gwaDWfSK;vmtc^W}S*+Tf*sVuH0-vk{I$uu(kA)Y(#N{&L;BDT~W%qT^5AOb}xjIS>?$<_;mVq!@u*R3>2q=PGhopot|&XaOTho}*) zC=V;4+&u}DiU>pCOsQ}G1#29M+HxGTyk&eRpVDQkT`dF;CWCj>X@^>oI9piJREck= zq%9mGDV-`H(pPgklH6P5j6Yj2E~{L|F^@zZ{C*4g8TlRKspl>a76dpdSr4SCFFOE#KY5XcFN_-9n!7w>p29Rskm`-er zw0%7r^iwXcSm}JJT)7aW19dDIPOv+TFuc-@z|#kxOdZz%Co9S)wfheO(pGbwgT$ zB2+uf;hHZFlx|Pcj%^DbMWHkpn5kQFZ%-aSST{+z*7P{TSjR{qAPZwCH4jsFKF(Nw zV#Z4^tggz#0Kc_5w0LpS@(s7HxanOa*1X*e+$nsGmfog0oMbeD`DwsxCeM zo(_I{X3!Dx0XkZ23A8NgFv2YG-wr%=uErUX!1-rSp_K+LtU5O=FV7Pe@l`T(${I)8 zES`Gpf`(1{jiV}uofQ}vx~OwuPv8j$ZAc&pb{u$GzD84K2sQ)pZ+0=pzP*HL$+VM!@!(h^O- z#zb(b@7{2XxG`G(0gMuMMGI^jawB*DM4&M^bM7d1X_v>j9X^BtE=ah6d7GXCHF3@D zHAPeQ3g}Eq2}d!#7OmD9A7p}=gBVXy0ZHuX5 zI8;yxDt!20H(M{SLfhrIV(MrvSTgkyer$IV1i-;bHEm;m3Ukq&J;qWFWfx?dZHjm9RjrYe z001_M7XY{%D55V)Zm*foV1#vnZ18LX7W$}leV^S(A6A=dJ{})Jcn&g33TENt9sWuH z9J2w&XXCA^6U%EMM4M38A|r=fV;uf=Csrd2MUhoG8AkIqX975g8gYNtWw)Ud6=+#! zN0W(adw8`76nSn)BVySza`yQ?O&%l{Gx7nbTI2DjHhC%?G{Wq60Hh(p2S}ogcDXE2 zV*?0aRqSQkJY?E4O2a19HD@D?r{Q6m6S!J$cYZ6LEYs z05C5_A!Me||`bW$piOlb%TSU0#-dzW$VYyRVIOc{|i zTvNje5wt^&LqjigRTWQ6W>hoG2y0yhWy{y0E+Xibbpd3ETzD_GIFh%)Cd#iYd4o^C z09A6)sfD5Kbl1U>sq zZy&IKchc#FC?GY`L2C-BYh*g1Pkustbn@pAL%Gjj6!2a$}mardORGPQ#P zZ`G);5%U{AIM6I%d+OxVbnjM!H9u`1F)OLO7>$nNQ|dIULv1EtRZU*uJ{MvwC;j7t z6{KBSJ$OZ4f2L zGU7e_BZIUcYPa)UBFI8jb+C`sIr%lqXHesI73o&bDTTXBaWg92P>3-{G+Y;WLkoNi z2;XM96p${~zFvrmeQ)4E~5Z(HOAAFGm1=5Z+ZkQgY zC+D3vA8*s@IF5IkB*bV=d*rV*Q6z~_L^4t^3UkluK5c&y8>GLid+T?nE#pv_24|$i zeTGc(7g^zNBQ&#?6Dk86ClOpNHKqvz6ewlbeZs^8RxX6WO?^XUY$U4F7}p0X2J`AF zTDUSn9$f@sRxDU^JRbXyCE3}wa}+%uP?y0N85sV)G?>~i0T=|ISso&&9IY435ZbMP za5{tnZ{hr9J&$fDLD4+&T*pBcYtP8iZQYDSdK=8|Ary;`SFVQ1B$20)31E=z zB7^O)JQ3{gV_!*xUzN*O5Ap9VX8lC>76gG|H$ac#NLRSx3E_iZ`zQP2W;o!A;v7leLE~}7jZI}TCd{vILMgse$MS3XQ{+O5t;GgVFfRQDZRDa zPx^oXYha*75b?=I0t4=~K%Bc~5PooIbT|%LN&d z6A~Xl$2W9)p{4;W__lw2x%>}7p!#o-dOvt5;Bk0ST@4PuXwNa^NV z^3Qn(r~gL{KY4$pu2Mcf=xk6yz&bqMz48|vSfLuUFZwvhfdB=`h_w#uW8)(vX(3LC zam)~^`Ho)&x)TPw=89+(E}2rB=tLwSQ*|Lv@>dSEr&b+dGv;#kSzANADCrPWw`&>j zt%FfkPp$)+a?Lp940~I%-iAy9c@Izc4OVBvNa1&az#ujbG#4O?@L3QN8E_ySocY}&bMUwIQ=ZrG;%61!iz`WCn_}5(ncpXU$HyWkZvtLu5%*wCgppKP$yxN zjxjaMU^-aN5RFT~Bo<7MuQhtbOBxy2+W~H=@rNnY&nzs$_!ufBW&jffY2q9E85v#4 zGpTJ72670Hu?AtvoHY{wV^I-gTW%k=`U70xwURf0u~cPT`7Ke5zY0}gq1h{m(B4-F zCf!ZpY{+Zff8#O0hGJd?ZK?xbtzbE7QGyC&=>r+a8yjc1VLE)OcHCwV+IkUuVIVhv z6u2T4b(b9~>ryu1DPUb0Orw60Q>bJlo-IYCX?+CF*c1{HjNf}Oj+}n@(OGk5u`w66 zhF(;o7q1}?lExb@n@$t12s|5Opo%=@BB(MMnLstjj-L}c>+uJZg#mipc5z8FT?|x( zib4=D&OI9J-w7t+wiF&@2qR5`bmj^*YuIJ9&(&vrxZE|yZ!UL_WV22UZk;lyibZ@w zXrp!U@Pk2Za1Vacr{oHbvU3A!GcFjlNH!X^j-4|XYT^=d16^jpaRD&4L?uvqr(!N( z*k4gCTm2-cEkjtB`W^?agEtRo_e&|a?6hE+SCkj4Mayzye>^qpreDDktE>y$Yx@@W(jQ!{ zo3L(fGcJ1O>T(fhiL_qY!;26ayek;-(lJx_+sFu~F>eyN2oOm~6Lw|XjKUv^8v_9V z#2ZXN=p%E|xZ@VBWkfKi4}v^wR!}KxtCvFKs)c)!5H@(=b=zzo3gA0n>XB_H#Cv1i zjPfjL4;}#mcN|TZpo}JW<~Aj4k=Yium6?9hGqx{PKqLVBELmmsFakK@xQYae)wC%9 zV0?V3N3@6F274bYO;$8#T=eBn~GOlwuyfFh-YgaST zr-2@^Qr}Ujj2vtyXVrH6RrPKkqQ!Vq0vRRMvQiQ$_>wD}|1U>VEX{2&id7=bHRoI! zPx)Btx*h|tQ)>_HH5M(nR^1I{7A8NnJ2nhFOVQ%wOA1l38a(^YgS zJZV6U-S7G59;aC1Ih)z zwvIN7$uN7&6iiLt+9DkMpI{KS%=1tDTc-&tV$wUWJyvFHU644p;!kc9z%>OEcOWl~ zlDB#ph;mBTaSLvhISg(Lnx9%4N#Y5KJ^yK|l#371u>B6OJ@I#aQI$eXTzC#8^7Kxr zQD-Wa8MG)dknDG2_H!`>nL9#fMfQG3Fp^<3`#)*~yCpW#AfqfN=(PkERAu=(QS1l_?)59UMwMqwF!pe8dN39*`NFG2$LY&y;$V z31=l&+BA4p1vydtGZQmS>{LTni%oT;Ivqmgewi%Z=+rM*%VThsH)jc+94;&d&v_XX zd`>LQJsDMu7~Wbb2GVWylvF#T`~4<*Du`PIsV6M~I1FcNTi;dXtp83eOru%KH+39x z2a{}df`Wn*8ono($>jnF)|=9_asY!3!tyj}-2GSC{#`WOrR zgF`+eL-;IwR+bOQHdSHoVmcvE1#KZk+aVC$64q41EjS7Bav>kdTuBe-p(=ZuUmAMY zk{2F5o!?G@G}2(k8zd0~DFXzI6A=}%AG~{r|M*D&2ILaWE%Iv$;30QYYNLC{5nO0W z=A}A*3eF=cQb1kI_(1`bO;{5L9-d&Pe*Q+Rm;q_2hypkr^cxS*h#OZ&2Hatl7kVeS z^6M6_><<$a=~xVxSFtcsVC^OwGum=BKAm-Y79dlfuL=iELZ@P+{w+=xB8Dc;u74aX zhMFH|y`U@c$;ln7sabv7_8t?^#!7dxkYHgSun~VRTB0Dfy_`LUh*DXc+TeA(KXxtO z@oy*^#cB`Qy3r{4zY9V@0g!TMkt#i=A^S7G+OJEVU*<80SY1r-%RmHi~JQV>d^-TlZO%3|2c6@suhIMkgjRNYhNhL&6+;L+df42p5i48Om|8SUvpdFNf&6T1Dq9T7emme0Mcr zIKNl%Se8MyUrfKnQJ8D%4{zliQ%#-}GdKRy2S@#hWq@)}IYmvXZVFieZVmw;SctdZ zKMC}nXj7S3GJWJ=IJTrzTANXGI*6VKU^IZYPIbU$H~O`Wa^s_}1na#y0IWLH0{|`& z4W|f(N)x&86<&VcYCg}dPZKUVQ9}2mGaZ|8UC{R93v-IKK=(GUMt@BBTmYY7Qehc% zH95l$5j{S_4%;+yBiV0EF?)v%7!^4gACXn>6sU^s3Riik8JVHBZBI6l4gp>yQ*0r} zTCtDa6c}(aI6S#fAUK062%r5lOWiTsE1U^$RiMEuVaC_wLl=y=N*J>qCZ?$724+PF zVz+X}YV~ zEZc7APV=JQe)9MF7(^UZ0AomATV;gdS;UQ=NJC)dLrAQpZ2b3Qcgpa&WLI+GTMd5N zc6LlCEmrWFXdR;TQR>#=J5ngICPYiQ1dv)GC$`eRQ%$XQO578@RE1y~E~2L>H_v<* zI{126OW;oTL|KudR#1@3FZk5vCWP#;7mlLN9NBJ9bN#FvYr(;nZj1`NWCCEF4v@+s zHpBfd5CHg6BWtw(I42P;O_5CkXm$u}N#F2Ne_>lpaYhRWZHWA_Z;21Y1P37wK*D{F zXV8o^S<{fuD_Fz(AF>H42>DpIMzah7barJ58M*5XOA|S|bHIvnCZ!0l1XAlTBFX3~ zD>JS{J$jkLRM0Uk9M1@~83br-AVwPYN$0x$eG2VCPLV0U7PFn6d?0`WPZ%CfcM^@j zM`)43CPd-mF5XZW#V zOYrA=c8_5uO7E|xN~XDB3`=$UNrRhxFCN{%G4>oYUroXDP=5RKV&^zpY&=Gt7@^xr z7oiT*FYdLy5yoDPX~E1eH~4Rc3Q{F29lJxk4=o6zET8G0QmVFgclkY^L(25CF(*62 z2o(thPDl_gPtQ?UIgwX8A4^{YbHL{&L_owDb^~`23aJ3eej6h^ep=iq5CLAFO?4pg zU_XUiI|UZOe8$TRGQom$C8r{;4w#goUc1H~G>}^GM!p0XHbV3@d3Z423QHabT&_j) zZGTS{7DpYeW(^{OQIQvgR=NAEd6!XKHkQZSd%X`PTU%R^9UYG@Xs|j*MrZBcOy)B{ zMn+Z|VDRmVX5_hVCY5-zCX|dwG1L0%VYT>8d_3WlR|<+ye`C$R1wrTC1|^vxBK)Wz zAu!M~V*9M8XDPE0DtgP`XgB_8dQ@|C7Kqsm9UeH8OD~i@KqnBvc6bFqB1f}GVCVJ^ zH+RdIOOISee3;yoZV@GgQy)F&5(wtFV|S6TA+5aic@h|19^Us5ClHA_Bi+eEVC>Xq zOl=d(PfOez9;myEad@8wTFbq76SrNZ5vdB)D71}*9OKtR zKx7;q8^&=*N<`K-d{610W~~tsdYrY?66u0QAkyM90IT|TYoz~OX2nv@P`+Vs0d;H+ zJA8dT1q<(m5SX5JOaiCaUIz0?N8+yxe?AG*MkL@`7d}!I8Q^PCe7%-qf9mYBW_? z5ZG9ZQj>_EWvmM;F0gIwCCi0qd8QdPKB+|TZF1YAAOvUw8%efBQYUzeBR?*+bik2Y z65!+-GhPWv5S~8_J@LYfdR!!O8X-oZMfib15E*-fdb!OaT+xhLItHOwPhAq{21)Jk z6@!=-LD4qMS4U9XHvw_JSpypvAD!fzGCu9|Iu%5qYt-E>Mi@7AJ;1(H66~q*Nxw&( zC~)e{OaM?70012yZho@^b1#n{NeaZ~GAefTH1EB`DoiIVV9!W3OYf#+W~k7#fA%HV zNLvz}d|BaXGLb|fPQXH%4_`DbU>!E(4uR23SYP9{ccl>%K4i$^5)xS3C{UHn7EH~; zd{|eDea*5nH1}P|O-ovZd*>JiWX2aCHC_6IOEp53OhUaBa#~|-A2x^x7yejWAS~GQ zAwjcNNSl3kSpVlRXrjr?M`54iY3)*(X`@iA4}KTq!T-R4_3;4!!T-R4_3;4!QUCzs zVPO6L|1S{#WBdQV9!T#8;`>1S9*DmK@o!x6_5Z8?*D?^Rn&m$LAx;2y|7wSwfz_?| ztBO@~)$Huk@`TaDoyXEk0|k*^@=z;s()uF|W8Ki}y#JmLShae8GUi6zKP3&TXIyREyM&6e*OIOk!6A-BC=2UF{(7?MO;KX(6_Frj^o@0FNh>s!!{465 zSB!)s?f+|XZDK*$xE5*4c!7K#A(Q+mlFtXA(wxseXC%L}22y!V|D;IE#rn{R$*9cf z?`+jGR?RPhkvs>rkbJ>InRikjT`%$U`vZyqp8Sj|E1~{YaKWeonUcTp!*F*4ob|RO zGWy&H)jX2PG1tc(_B$tpEOx!$&SMu1k$L>v96xPVDTo$R_jRrAad}|g&eUFx$ZHE0 zh?1ly+XV(XUggN99CiDVyW1aJvBYoXIPLtQomR?MJa%|NfK;e8<^fUJ_$Wa^HyaA2 zB+?nX1{ z2Oi(ot7PNJ$lJs-M`#@*l2E9g4%~f9fI&=PEV<%9Q=<#=9~ zSLRzfL-b)dTJZuOo#%BzIDN~-p!30S%WqkoSbC(!`9P}iz)YIVy3ckWN3=PX%D-OA zem0zFeO#lWxv*}v0s)t#A(~}~z;G86xHWWQxPIgeMHL7Vxpii<7F;8rHLukcAl2lPCh)2~OB~)($E23p_a8!Ao;Yr1=o8 zh7!*CrXoe%yD0{XeETTtB#Oj*%1$DRBY-#DT++Lz%OznP3YuK3*!zFN26BQDR1Rw7 zVSWr518h@I3|1TX7WnndYE+yH7AZ)?IKa?zzb*}HOOVE9c6TnVa5j=lZYLzGJXWnC@GQP5*6Wlc92CP zxHT5=u1r(^`vs-yE#g5!O#N-XLhmkZFfC?PPSA|QX-s}R1)kaHO@Zd2+G07Tw8Rh_ zqd*6{zoN)EDsgMl9--&r(X3l5Qe#0PJ3Mt;>7`JvxG%_J&X&l{h(gYLjCR0~Z`dn3 zC>ZD)PszJ-cVED3AF+qA6mz4PYRl}P3|=Wn&9jEjT~4S(HMnzte!Hi@J#bz zomLHMu1dT%71?8&ud=m^LSgja_RxHGHkvv=6$Av&dv{x3ZuNP{T$)=-5zt5XQkc5X zC>~#iGWn#`aMcfxC=&dx>>edVU~}4L!IvZ-%mbYNo#C5Uo@#TlwW;oLssm;Q z&YVUpP5(q^uXPo_`*p15qM?#(Q|R|QJ8h~^q?c~C_^A5D6fFK?+8b{0t=y%r;HbPu zhV0hs?|46Et{?&RD#q+f8^)N2HV3f4TQsRfr1#;n`Vdzv@Jii}N={u??jW$YrpLRi z#;d+tP#K8YM@Tm3fJvXCs77;~;&;LVl%a{p`+-PP3mRFK>ue>XG71cKHWmR>2KkS& zH*vcFGI~gnFZu*C%S*X76!qLkYTML98z8AKVpPoGGN*%8RUn^=21Hrx{{WFN-fbCE z2uZ6g2iJZjm_-V+y}e;TrV)W(%5S?lCyK}TWKTJv2VGL>b3NfH5D}se1o#G(k_vt= zm(=b9NB{sbKWN|`Y_M&nqgDE(M<6$;XrU0|s^YIbt!}Q3AGZHKahnRo9kMx6FC;st z`C7lQMrG+p6B8uABBdAC!Gfr@-QvHX;bxaC1*@%V2jlR4pN&QG-6&GeVJ7C61ck^K zSfZg$E(Wbej^4NV7N^f*pNNf(CH>ICPoWg7*xiz^&+*3Ao$53g5z%mYYEEHoDMJin zZasdul`=32aJ|TR@2V0nrQ{Y*9=aIkkmZqL!?&JH0h#9c^$`4Ag#9LFJx`cn_{CLA zFWigSqPXhV;+|?zToAQAY^^F=+hl-qcOI-D zLmuXjoJs@y<=Yc==y(4o$s4$Kznza5Bd-%I&~T2$g7^EQZjSy8$iw)R9AqSwbyySX zFK27*ah!zGq*G3=&D-lYk@^>w7WFqM?}F(DZga}&+1p9bqHT7$KrGlK=&d`CS6}X{ z1`j`xNuZ*7NHmW+D=(o>nibvyClT59ejT(q!!OQC8r0>lu(9bAE=oQmVAi^O9>b}` zvbGqUO$thWu@jXCdVA@*lqn_}vV<8r(~e!`&l>wO?pcKv=t{bCwoS}Q0dxSge;)7_ zm|~~bss@%dJ(`+~kXZVT;Bf`~B(>K(##`C@zUbjr2@ANdRr1i`~ z@Uq&nTGj4&5WFz39e>#B{R0<{iZUtn_28(3?&3)pgpQk@bib}4Fh&>SkuIT0=Hu>+ioe* z{*T43yP<0$pv+)C_Z0pnA1C|L3a=Cdz(sGtQpqTT!@+qEGTBguMN(x*PCkvQ-98i6FKenYm`B*_zn3}18E9S4^GpP_WT zWYjT>CUpzpSnbMgTwl@wPhfPT+4*^}(^j`|6qR#pJDlgPzwuQP+qORPe;GdPmjc6w zYcqHHi!_rMG)BiIz>U(SC+Y-I!JB#Dck8xAPu=J^Sq)6cD2p5-5MM{0-e@aD$9p44 zHo5bmDG9>`stC-iU91j_SC?FL89VD2TdLh>E$``1kdIXe(HtNs0y!aP%-xhnSb6K* zmw4fMd7%|m)nR8dSD0n*DMj=&Xi&VPx-Qcd{Wk4K51^5b+K_y*YN4kL`Cx+jpu&{Y zog1$GzIm>muDt8g%(|Ni@7t?y=nk)4ebDmGdcAyW92fhEG7sH}y~+4WM!OMvkyK!-idKZ!nY~cm`SqWA}&CBTTqK zFYUQAC6gPuk75EZv43?uLt~7pJ8&NKdK7U09xbqP2Ew$X^ILdqra{rN5I9?{ll?1V z*z^3%3+Fkw?8j_IS`Il;XM)OJ0Tmub`kZFWa2JDcNDM=@Y^7%6=XnBf*%!w4Og4%< zB;r2ns{oKn!r7P!eu{i$Rxv*10N*t1vN0`oCnI%v>9hfS6bHGyrV)lbvfu=@yRV{r zdhEfvTSLadNA^8-kb@=nL6GUX3OyGF9_l;fU{G*;3){hLxj3=gfWV~~-5gRBNJ9&A z<+!ZoPAo0x_;&5P!|Xn(DSp_efk&FOJ?HXd6g0oJ5f=sqWp(WIs=|#Ur-d*ndo`RR zuQK*=ylhNv7TLO~zQd1+SSN!P)%N$Ea9!+KIkIKmL<;x82HpDfTH}-=&%&~>FvXC; zAXqDo;}FZXcQ0cIHldbXw(Vbcn_Hdh7+@=dB>~>Zwt%MyDG;)xIbO~4Ft&9B^JW>* zOg$pGmYC7mtNF;W-AAUBv+elTuQW{rdUyC!qV|4e1$j0nwyIjrQSqzt7%QtvkDGkM zMfW~ar%xJ(!U)~8OiYm(*M-4e(0$L%&H{m!&P$^G&-TDg!@8};rO;>IN~HfEiud~< zJO;E~23IqL;UltqU4|P*aedJ!U~X9?MCW%4$YvtF*689;qVjCE+_2v5KHes}_#~f) z+$JD7L2UIv)M--S93i7J1$GkC_WeGQe&9QAP>wt7JmuiK(<=*%o2i^Qd~$ZRxwrt~ zEW0bUdFseaw-KRWXtK%%=dR%@#9Xy?rV}J#wNeEGuS$xF8O#7{!Jw&zp>V`fIe0&3 zdH3+z+`b0&uV_#r`LfM4PJQ_=_e%eyk3Pl@lC*nPSkG#1W=&>n@Y>evy_|xD%Wga% zO5WP>CNMj^bEb9$iKija9h%QdpgI$dZq6J6VkO2ftao|b%p5sA^`P~vgp@~M3jh2m zKEHO}9Bp&^_#&2m#)&cC10O9HkK*BNqPJC-=417ZyQCRhsMWn*l$2)BxqDY`<8%Xc zwJ!FjwtBL$#Rx#e*>^6~ZiQ)MDN(YN+ZBJ(Z6?{&Zuv~ z$~|T1$bdT7VZizAHEr~Tm^ql4EgkGG1%_$2oQVFw0CKSY9xS@USR;z?(o+$ zp(nhuM@hg3x`DK4os=j=ul*?v+y6mfRs#kxvgs$T#~gNJx@qgURr5=L&Mi~{Jc?|X zd7*o(zZ-caQ`4lsOaK5I;&h`vI_X5_E>h?`;GeArbS&7FTn5i8lE5Y_xo-#A+rfTu zk{1ZPvHhkFlK}*b2jP6a2_W4If)B1g9uHkbTiw3I_M>F0_3Ec-hzmWgdjl0zp z(8a1LTarDkKSj-Ex&qx;uQ#^CrgXI}2KHi4)!W+ULJ6MY#{v#rOc^qrp3su>K~}lY zqP^hSbIGE~W7wnbhsij;>4r7Bt((lvf0S8u=@kq0W_WlggCh#F^15JKG!iMlyXJ{GO9{v|kFbJEJ~NKwYIs z2DY!Icuvp@!|fg$Ts?Ho+o>&k_ZSJw1BHE4b-ED8#5i~~TdBE$*`u+ZQ$Y*{p zxENZFIMSsxcb>7}`G+S{U9Bjn(t0?m_6XbGkq$o;N+-eUJ&8Ul-X*Yi2slLM8>cPd z!TtLY#bnUF<+Bhok+mk=PSTRp%mQ1Olc80AxBdyIOf(gr`Hnh{>GtSUm976j1qQ8f zm9%%u!1^bn`v4v-uu#Vvf{kWSa7?I%#@X$g4~zq{$J~9EYkqLHJHElM#cOp&5*ap7~QLQkYz@{=%JF zKwPZn$0xBcRuTtP79+sAb`M$m%(PUSTidJ-+o;9?qH$TKWF?;KRJvu9Fh|Q^3{EPG z!T6t9DBvZP)1;45M&&irv4E9{ljnzJOIqyos~Hs|NGdB?Wc}IpR&1zQR2Qv>EPmub z%_bnepKE8+ReL5Rxv?1wLc@4tB5}r+Ve;G4LzT}e)9KqmyuTtWkeiSAY4m`Sfl_S1md5B9uWmx)O%$|5972N6=JPjFsjqc zm#>`OClNXm7@hRZh4fn#Kg$CD9P<+3v-4aomdkF{uEDXz$7_VRC!BKzckwmC-^|CHIm2{d)e6<^GPO`k zmk(3*j$8+s*qLM}S9wtBWE8iH#(FgZ&Za^xRiRF4I3ClHFzmy}qo?v=49(XEW2R+= z&Q#~lLrbmlStJzKSBSU>2D1yQ?Me2nvZeixAzQAoqZs0Jzs|d+K$qG)}(!+$r zKw|xtSw3^gx83)x`u6A9+|--EhY0Qf4{iVQHQHsQ87b;Nyb)O{xCn9GM=xqJwu=Mm zVY#bYe?(>veEipCP1*6n!9KGB0dl6#{w%ET`H#dG%HUq_%29FhMLgBf39ybDDAJ%O z;@Q+KP3F;1oS$1hB(uqwNgI`d#1#qCzgWPb>G5THusXH_reo8(@v>s00>`HE$-yv$ zzo;B+Zp5gqP6S93!Ps7NX%L#LAg{`G zDd0+qT&@g(=zD+s-5W67tAw+>3YdUOplHKDF`VyDL|F+DNbmuK*)F2A+<|qg)0oLO zo!(ViIRG9lumZl&KP<~Rl5mSi5KPzSQ;FCd*Q@7qva_3RSVeMmyLX!_H6_FZr3?A^ zI~eBu!FTcSF=%l;p;V0ArE=1zKgy^Mx!$T#(SadJ{k5O&+m)z)f*KWnRrA}WwEH-z zZ&+m8QWkWxs|IoULDlW5DifJHj4%?xCk&;(U5(+tU!Zj{jX?t}>D2|YR)h=)AElpW zv)SLiCug?1F4AcUd4(3vj|Jo-wRzr&=~0FX+fv}Yokw_;;zyS&epo(Svm53)zm$|p zfw3O%N+7~=!xCX(tAogF?@YP+Tm`}TCI=4!UIY8EqtVd9 zE8V+#loOpZDrMP4=wf> zSj|tH1UEg7lDAh%$z2$V?!)gtmj04s6n`_23Y6v=IJlwqD20svr;1nk|D)dIvF`J0 z*o+<3Ljpl7f?%Uz*`tnKdJwmn#K{~)UJ^Lz80waO5t5*D0cH(H{=q$4JB0P2q|cbA6;W!nb04!*$#;pkHiZruGtVzh??V6a1`Ng^*| z7`i|L8W0Q5cgI%$Id4NrH1<49cI3lvc|JCIZl<&*1`#+AY95X^Q5rl+sweFqwS}C2nY%i&)n4@=v z3e0*|U!A1JVeXX8_b(^EbIKEJFp2>M>Lq?+-K>Dpn8d>ezW?}P&%p(-BF0aX+ioJelf9kes|0h`4Qggt^RBS$+?9CN=L5W{X)QH6}}ulfb~U* z_0oO@a%tz9eES!)vAUz;&eps_h;zY~-%1~HgZIsodKZ)p51v5lX~XhHqjVT00b$nD zam1uE<+}M?-O%(>j0mR;(OpH7(r`xzh2gVFNl4tq#NneZpNnvFFeE}|Gns#Lb5qMc z9wh5BHjN^()ef4n*5_@USPCdSln-VHhCJ0#>g!zrIAntLd@w}ww9US{2Cu!no?n|f z2&S+xteM=a3TQwtkOM3WR~Z%3?oH&y!;aNypx1+cvEF_Y5A}V!aG7mBFcaXB5NOh9 z;YV`@&PLQD-L&*s!Gti|?lO%z5Zs--q=pm?+s{P8LJ{Aw-tCUKL~VFJCRZ?LCD>|V zXh+Dxu88U#4#D$EXoU~hkt1>k{PZKPTD}cJhFo*zK#bvQLMPd~$E<+yjVw0VvAJN1 zGWi;JfU@qj%MNPZqeJzunO4OZbuoRnH$@U;bKC#=UkX;-t?E~M!I-%zQgX~yywm(VHgImR#Y?E|bu@30Dfufcce%(mgJ~zpk zL+xzd!-4079D3yoN4Fcxy>h&C4*h)J%?R}hjAgk-T<1%R^1V%AMcOd)Y$PAl_|MaP zU6N-}ab87{{TzW!`N`72#5T&Cf+QM?)poxqRz;+9&yf{`euG!qCafN8fpEsJ9x7%4 z0=Hu*CHCqg0-5+3py9x6>UJ}ms}dKMAcT0UXR)U~VQ-XZAhS0zu3nV@6SPlnN!r8* zOg%1|-FLiVe|pK`L6@XI#gaGT{9R>FKsb~TwcXf*Hf}lF+fGWaUesjMW@LK@Pf+Rv zWSf|l5kzlgX-f78G0*chbJ4<;L<;w5m3q`?dWF>+OF1RMlJD)%15Zc4rcM{Ve3?;h z$psKj05P8~x@>20Jit0EC0fiZxn8Bg|F@uzhzOTqU zSg6m!wLBFgUD?S7NcvJ-g}2`_kh=?14oB{hP}_O!Y)_ZAv?NO@C!V72zfgdTq+J35 zx4%SsXgyZfRFM0>od`sZ&bgJmDHv9w!X%jyBIrL5`hj_1k~{veObkLDcvkfQg-Bh# zX#gBLM)_7vf+2u~xwOJS{Ncs%8p7NwGs$8GV3UUWI!b(WP~aaGM@w>o9tK_fyjz@9 zAh`)%m;b`*4G?qw@5LTml?|CgA|=IXveHpVlFPNCPFHXqKF0MKo1sjwbyok0CP}*3 zc}iuof63l?_*Zn~s#+{SQY%j)sYd}-r=zNW zrtCV-IsJFMmqCix$Xp7dR>#SB6Izc*bl-hlhR-C<>j|#VN<@1gI8dmlOr6yw!LnH; zg@e!ygL4?-mR`MFC6-_Kus7U<@+)DyY+_ziIkIOzKLA2xS=&qJu59DNQe%(Rq?1Aj zA$Zvk*Z=@BKP{s^ci4_yjJC#Ox9bNocz)oQufX7V(St+}^|Gu}_agY^TRk3dIQ(&v z9R4@m317u7k;qOW1N%K3!K}zUt*E@2?n*&$?XnR;%_MS1BH0 zr~hG=NEJ2$3tnnwEr^=uC7hNbhlinq2e*9cqHUt+x6RK8oPbBs=c4svlkqX+lG*UQj+6qrzW{97BuY=KN8OVqJ9sctq-@;1^_a+ zCT(y0Flnd2v$bTks5=nd&%=~s^XyT4ihR%)=TsA$ZGySfV>pVhwQr%<=ZGFxybq2e zWIjz66-2)O9)Gvqxx?gaO73%>l8-tAE8(zD*3F)=V&;)+V05qT)tFJuhHkg{%LL+b z3T5V$(pB(~)+Ioxh)iK;_159@R_FkW{g?tQqQ>zLyN~9fkV{PE0Uepg99OpJ(`cx= zJD7A&&e%U69@DcabkhJ)>HA$mhXV>uFx@YGF4gRoo^`)*XdW zt7ckUn)1#!S^NiS=N0M$vQFW)Cxa?b-t&~bXG~OY^zW-8g3<*9X(L5OrFWzZB2Bu0 z^eRa2ART5XA|TR#ICKQ1Nf(hmfOG|h-eKq+28J@j)WbhHFHTNwZtmULS^L#a_D

n9@$s5b4$R|Pw9@R z`@P}V7NB^r8QCumkdrHnhM&laBvL#61DBKcGr7=QZiBjHjOZRKNr)?#47kmy*u2kW zGIJWN)~(wRKj8_VJh;DpDC=7equT$Z-nanqEM^5?$e0IkNh`t)nZ8-nGx&rK%t&VU z*)RCfJmP*pqtwGCeqkrzn@p-DUKF@&QuNk*+FeR`300wd7ouQ%1}~wzw3lfNMp(Tz zSXTM^ef74-mxY^75Bod)AgS>!pONi%J-(k{Lc2TjOp;DwM*=sfn%QW${cQzmE*w%V ztDo2j&LF-AA1+5EW-cx2yEH!_kDP6#N{tZv(%R79a<#qon7ef_$B&`NkNjdq?xO#f zt$Ktd{i}DhLaCpf{So$LjWq&}a2`{%@u`a^j*KxIdhb^slEGRkG230Mro5=onWG4p z#oXfi_tuEWOX{x=fflvq)vs^4u*fgH&COG|a0(j59_qiPJ-cv9A+LymGxDCqx=z$OG5jt61K&75YNFu9mKpNZXLZD?vq3cTw z;fVw9ELv)KyklEeHHo9Ty*8JXuK951f7aza=x{ffEV=Nhg0s`Qke!>YX$A+JCRG>G zW@PvE%?mO*9f~gRt-tqvywirUQQwj7qi7snne3+>;`%6*xntlZ8!DaMSG(Cq=y7nj zzHi!KfAh$DeO4$sHj!p)l-XJBb`| z66ZTp$V7hvy#H9u1Lww#jr!Tk(ET4G0|y0TgS3r&T%#_Mms%R}=t$=p}_LF9^W zK;8>;D1nnKpw@!byz_dF;#~!z?|a=W!C-jA5OuEeDWq5Ew1Pdy%QZQZ?+`3;>p<5I zu=$GHX&(7aYe=B=cIO!DrI4x-N0AzCp5p+>BVotbrI#gZN+7in5K;pD%d>D9`q zue4QCqLw25xXJDrvMzLS|B8f_zG{mkM=O7dvC~$Q^$zI}fAo~%re6WK-pg78CXjdN z;J%oR<@(1+jNg}^4#GaiP?03()T>Df)--YKgkVQ-e)zbln@IN6E>t zyuNE{@$R!QD^s^ejyEVyxpjQg=$BqtjV5%CF4*P$vS=Y^{ck{&&V+cH_pMTekJv>Sd+D=n4gw%0%6p^3ch&n zG*Jrd{sv%7pN+|Ofam^-Oduy-eReN0h|Z&Efbgn|ZlehQQ|tHb*LNUS1rLkaC8eeD zz5UgaHKh(A14E|~zi!%iKiE3Rkcd`f1?GLGkjRg;_uND#>@09$*%}=v{%(^i93HiQaQ9 zhRL-y+Mrm1D{Wuy|7;Y2Pb-wkF;$;zm2O7)5}GY4@ae;KA}c4K8s zMeOqO#QNj_C@!b3g);MN7<||_Gsi||#mdIi6}jQ2pHSZ@c&i~!G%z&O(y0L~6j1eK zSnX>}L^LAGhsi#kwpRXe9ra78yQJgHke?|xC&f};Po(%52PUnRnHH)ahiXnOJjJyS zr0~F&bGLirBRk|q7Ojs3%lKNH#tqYUoA$xd9UpWh2IF|6jOOo_bZalAF0RJu zQ(Dje2?Tmkrv3N{ZU?ckOSq=wS}>V~NFI$6sH|&LLQY+{%bf~-@Bw9ae(rqqv;ML9 zQR|^&6f(a$bvEI`BwfYG*-x0enB2x$55Iv;v!4m5dsZ_d%Eo>3Y;Mld{O;k;C3d&` zNvE)O-7m`u39pDP5S(@$(gs~ zXSlD0&|jS#lf&@;KbAJ1@83AesrKHmdJso*@7BFrcit>Sx-@;Yi&R$buIm4N@7|r; z)U2=RU(?g4X5L|He^ggj=Rrl)<}o*?KJ8}u=>ZM3prBwl(jV>+Li}jVW8HE((6NID zCFgHE=+idcg#Uc{Nz<08b`Mu0{Hg3$e(904r_A1*gE12zNA^ix5ga@10ob;32cdQI??{1-h3^FCDrmCe z_X8HNv(1cossF++lFGtL>1q1q(IjSW;8M;LF>$GwgnPe|A_*#SFLT9r<6SvQI%@|@ zJ|P{kxSuJ=GvRH*)Id&#B&^JLMnex+qY`d=RiK`WYJG*-QzlW&0w5FwJ8OpSNAi5Rk!!AuOHbY^HpOM z+(8a4>lxHL*>AOa44Kc;Y>Otv!}uCWtpMCPMUtx=;8jVU%s57j|wM2&1&)|WGj{6D&^fxO^$?0DS-X6*1?D(nKeuw#s+*Q zwJ>LTDua)nwmCfAqwy_%QFcQqX*hqv;hfV!z5`Lfj0EoKWzOYKH8SBq%@lYDD zngqe%=ai>E>5!`=pnH!_el$B553(Bc*r^#)3IyzT-D*$dpOG`zFs&I0X}R-H@#0E@ z(=4i|10369QcbUv^=|3nMbXFZlV7AoTjHLAoz8IHK)i8>*Q`hZ{2i;trerqYT+Cel9Q$1 zWbdqT$(I&%$JYK)t-M$ldsPf=*REBbnR^Mvy$Ee9yAToS-9#K`b0kFwEEkAHnDgEf zQ!*-K@94T%Vjt(kt~czdhY!imrh_Ema<=Cf)#HSi)&Yx6YYXo&c`^UaP-suJ&5g=% zy4n~rQ3`g^@*k@oovH3zrAai#vZYGd^jVxx)xxm}7O`dJV!=sr>g_(T*X}7ZY1;K$ zR@?34h+h4CvRvC4O3^JcV9Sg=puXs>JskakAJs2lcN?@i@pn0G=jZWC0C?9~^7hP` zu!6;>rwZw_8B%w25}a!S%{NRw!>0h~LM@S1-SyW05G4WShDLThY>Zc<@LS9AWES70 z#dl!$bhO}V0(O5D@?rud^mTpIlRiN3YA}iVQsUP&iU8_6b_sMRcfs>5qgEqZ&SkADq(2ucPKL?dgJf&vF-+`$lw}mxjbDzP`__^p=QBeY}0gf9J2<>vd zdj9Lu{CiT?S{Xg$gva@$L!vY+;x=sY846^oZ;58qE{U71fkH$i4bC3e2W(5tx+Mox zh*V`jb}rKdf_D%Mp-FOg^;X*%;hA%5Wrl(h$i`07%{A5Jv3uaPmfPTp07j``VZF`B zlu|dZ8e%u+a-@DuHcD5qo5kh0tzJ|H-r*C~WbR!J?xqlm=5FaCZ$GuVN$X?K!haD^ zno^35W<&82OXfyLvO`WK3!TNHYMoOcK}l<9DjR_d@fO8${pNKtuc?#)EAR7itk9My zspc@;%Z;ZdL4mgO^lH=Q6+0eT%@PhQO}g;gY31lCn@`cT)AtAqh`LU-(k9R+sVjf# zj`oVaV$kOv+5d-|gcds(u#qO*3o7)E3|EH75GgyD zv65HIcFY}dOTfx+wdx`VKOQ2s#$y~vOF}V3st5p9?}{^wk-#3>VVSZ#xA;{_;k;ot zYAmc2KCmD&g3(I?oln@%D?Y(KJ0B}p)(1p*c?)#fk{)(QW953n3S(z!k=Y4JT#l|O zZox8gcoE8uZ+l}qPuE#z8I?L(9m3qylwc$_?7{h_AE5EB*KURGp~LvjTiVcS6*{Q< zSs4BrbiL3LJ_Z%syuXx9VS~bL3(JlmBXM2iH7&%UqkfKEU`Gz)tj_1mAfQhk;_>b#V5z}TQBoW)(pB#dW@sqV598(1=+Ah5RJA?cF}dYLAM&m;tN;&^OqRPxMNe{i{YwaUorL(R`59ITe?p-&@c+V zqi5uP^Z6>yG$4A#uMm8q!GKAlAh-t)P_%VZB_X}rP9dIi#bPK&Pf!WoK%*-o6iR|r zpgN*0@+yTyVQa8xDIv3l)a2;`Zc{m_lH`WWh6dXs8xUZtf~%bAzTnz|CIg^IGaW9Ectt( zb&LaajUzbiIm&z&OpIrZr#-QdgG#sbW83c=BCZ}2r-IgPngemJIumEca)6SJ9myTh zd{Xx4K*U_I7Ab6RA)pl^_d)28q(l1lp4f#ym)D!6oj8qP3}kU#DmiPAg!H$+hgEGn zOa@GvUHIbjYJ&DKib*LT4}~({7{dEbJ>R9Wjl#W^oA5HyvyqBPi5<`0S0DuLr2Z9L z7?N_e=^INEKINN)84S@89KKRkASgjd@jG3qVOGGJumU7cjVB{UFqakQxvEp6;o4r} zegHaT6v=W(k#(MW8!rnhJ@ZzvQwayUwx35WewGU`fM=V*WEd8IBrHAHe$kO!B>_iX z!!vG~?hii)+2)pS7_%X?lO|5B*|QAk>N-=Erfz?3xylye<Ld=?Wb+So+ zVoJS~;F3KT+Iw@$@UwsezbQua_sZ0131Ur`dNr{#jib26PJ3DY`5lezQO<`fS=}9$ zQn~)7>@J+Nlk#iaM0bZ5JBv2N_x4L4q~ifJ)zi2ZxyMiy-_hVf16it3yTs+BkW(wg z+eF?|3YFui$sS<18Osdn9wNvyuO|S@exdA-S=vv@S_2-ua4f{w>sHE%hNK3qs^_oH z&n=Pu(>I6Pk2F?ajmPbhb|0ak!l7FxHZhC`6wT8M`Rt1pRpSmdmH>Bngev-tEU6ZXuzDUcFkW>jx|z?=qaKf195V*3cn|8K;WUQ@r49R;w}wUp^Bv)_^C`?%r<&j zbg2X|yk%mp(v{}k2rIBg*ZWq|$*LB(i!Ba1H*0%$(S?VKld{e9rjFXA$l}_Ae2=13 zPL76i#u`>dF!YEL>8|X9p#CJ-8CO|SBNf1IS~&|UsFiN=wdtA28y_&P-d#g2tJHqbUx0Dwbl6&evNJ9i5v&S=`$_yqFt-cGm?0N>1$3{khpC z*4uA$XvODgVI5ijre=8UOzVT4Ke&?9`;6{;-#}ekqqq}_a4xJK3|t6FrD(G6MgCf@mf8H6)ZY0pT0d_isJP%CsHO_--xBUZeNMCeN94F? zV9B9%^~8j8C#a8K6jCqpN22+*ep#U?Lbmg8c^+Cs(;wHt#1$!_n7nkx1xp0!D**?p z6`pi}Jpl47-fmY;-hz1w#9nrfoghgBq6@78>5U8gK}f%6M z^tMxxS*xPY-z?V*Y@yRZqC>b7>12`5G0y+wZ^*sqq?@6%>PoeUo*r&yor8zz9!QH9 z)S_T^pSUW>35nP(Dc^X-KWOpV`J=?YxP@13N+Wmi^%ACW)HBRJ>_a%kMtK97%w{W^ zwvMHrwA9&6|IsPvw`0DkpR;er7z4&|g`(3@)&Y_f{)~z6sh2^I5 z1ngU(F8!UA6YJz6M>iD@hY&R`oFCQ-JRa1=B(&!54hWO*4mOiDn@)qr`ZsJtK8S=) zs3+ZrClLpsU?|$>T2nc*ZXguNDHTg~)lou6lQP0AT;z=ku5q9t&zJ{VGqBgc^E~ga zuCH(K`eQjGh9Z~v1%6CN*O$(Lsm3jHl^*tVul4TKr=hWzfY87zuYQU1W4f@#VRwC^ zgb0M{{ziRp40#^RR3w{4_b?Z})twPCy7`dxkN0tpGj^aN5PIRI`j zW{|gv)~dp61ugQ=oTGH3r8cK+?)KyQ^U=ROeaa8z&%g+)a=?-*6+)rl2RSAOmu(e= zzPcu9g~uI8xc_qI2tiu6%?u_r!v5Ms*uqR_;%tIEm;a$-^_B)ZGUOeR>cUf@o}V{k zj=R5wTgn;nW)hnNdC+qzbtb7)m)eM^os&yL>oG{H-2N+}-Dw}MZ^vtnSEad~x~!&G z+j5*lGQI!LHgq<3u6K+6X-_LN4V2yYhrW6Js*=Yw?dwbLBJGb^uPoG`FtcTLe1G!9 zgF%-3NXN^upBHeX=@@7u#v-yelDjIVUHT8 zP+wIC9Jn`FpjcDOt{Q+Yl{676JcYIi)y>yb0Zb8W-9I{{WRuFUM>FUG zb`_;^4aD9wx_-1KBysdv$#K@D?F!Ae!uxdF;E@Hk8D^TVJ5tUsaE^-nE9FX`18(xyOYHc1TZSCqnGXg#=D72STFDxyK&*FQfGFCZ$27V@KFaB&|YQ# zxEKX(1c{NOL}JℜQ-zJ+fC8<2X<@+u|9_ClO=;^(Zwp)bWwcRY1-AVrT{YUj9G* zWsSmc`7{(+$pP5t9OO(W=xpY^VVo24W_tR;03^G?ymXuC8I{zYroKaFMbR?H7myHn zFE^iB@7t#qw0HGFbZ(*E@>tW1G0|u9sFfu8&kER#(GOHj4coY7Z5vIBONJq@2zUz- z;=Frd*hfV$8gwm+HveM29cW3yRW5A=zv!HMY;m65A7}bWWLX64)8f)(I9_HJ6xiWn zRk51}zv_627jJUOwI9!WL-4Zadfo80u%AjoipaY&^$4-2uiA7x)*jbkfnL8k-Vm2o zsoDvb=gawlsOFdNYg)~l*O0VnCfxMaE4Ez-9e=wh>HUd8vGvt?+p?T@=4g}LwrjXg zri)yl<}-T0lNKH7z+mS#aMOBU;i{E@?Rom9WJgKUUdW5H(oEO(fb4#KaZuT-^9$bM zA>#$-iJf(Ve{bQ^BO-f0+54KI9{sJKP}fN7FGIDZfsaugRRPH; zWIh9tj0~KAZ}%~tzUar=#!$61}<;@goF1*L3%E(lqdQ_&2Fe=D=!h;-T!wf(9JW7@YWb7JqM3@DDQc`Zr+b z^<0lfH;4%~d5HXA>KBC(b59HLvVXy1y*<`@?xksn#lIIg?bV)JC+_cyFn10m)KS@j z+kl%a$=+kgh_o_-&A3wC?EOP71tYwGTCry~#0G1I?nsW?U|S9kjF|dIhgr1z<<;k0 z{B*D>4Wn2cEdIFn--%Fh(2PsFJZ90f*+p&@UN~0^0tbF(6fyy4_w}YVy*yK*UA zP}+4k4f+zlVz^>K)162y4?3~j;amt(3Z%Y?CcD>1u@Og!s%8og&2FY<=F0iD+P6BA z(@C%%H=CKZ`R4y}5DFV?yaD+&pcf@geCn+$8*W}OBf_kgwRdO)#nNw1A$ikyY4U#X zCnuUDBy$V%7ehsefJB7uuggNRf|i}ckMvLpx&-|@araL&1TV(p!cPYZztsgj1#`a0 z3+yY*O>MCL;=#YA%}zapqZdcaO(Qv2n1I`+=R-F&+HJ-|FwRPsda7JxE8Bl|gH{Z8_BjPp*HI}2+5EIBIU`6tH7 zs12sI^$`v36pb<9G9R0ybr38>ukFR_XLr|7)CzkJWLK}p`SHL?dSb~ zo>@#s1e+nUBf_ef5h?hrVmW&dA8kWBTTlb?E8jTwkGwlQ!Cxv+Hoj1+{y-9sxfO^e zZ+cnIUZz;gVse1CU$5IxAG#*p z6#EsyrKVizs+CDzR~^}d!3nJ8O+T~7Et!{wSeFB=E8N-Vf*zq_%4Vz&S9!p=dRsc| z0LhN;?k&3CiE|sn#=3XS_zK;Ik>77%BbfSl%Z(zQ{Q!=EtFVT9`GoAqH z_cGoz(mtHH2cam}m(n{w(7+q76~3Lj@EqmT;=nwDF7JfJLj`*C*Fc@OI|4J5bjKH= zvI2K(L(}3G1!Hra% zZRLe5V>sac3nP>rqpWl$QIvwjp<0kR*bsRQe2lOhC*hb}mOW2SwXq)^PGY(UfO(R1 zV4(g3#0{nSh`{SF?2O)FOsHk=O-T$J3&fCZD*OACd`8EMjv|z;QvTzZVH$E*99DGX zh=6nY{Jydv*d&Hscpm;|-&tli;4s)RQh8NY4)t%*xKu8XL|rjqy@4cjh-57`!hqs{ zH)*@DyO6O9H>{e^q78;o(hQJ5nE9N|taC(X*?#ml<;|W)%ZSg6O6q!>OQPe~WANKH z+dV<}SEyvid^H8L<*!A>G64v&f8{J4sW`z95Fk$%h)9N2gb1UE99aGgx^1aye|nau zb7fM2)-eiw?GBRUaIKRtzN?kE&=c;{yrOpLSj-|A*xnlvi+-^&5YZfGdaiKU`Z3}< zC({b^({P}j*$MgdfQRy}EN+N}Vaxo-3%1x?_4BlEX$O30u4AE@&RwIk!8p-TTh08r zo1GMj8VqHVLr|ZTUB=ZfD#7+xUz?%`$T#n5y&0_8y3{SD=4NDgPsr(KTOMKjA=P)=Y**jRjrp-^SjbF-FA2;!5}BPDiX%@)8*$sPmi4#{=jE;)mP0wzM@&O+T%sIXMYBxE zATzvU{r2f0B};XoP|jVD0#!2k#_Y;)(4=WgT;fczO$lQVY{~!cksRp3z@)^hWQa?_<}HAsWfW$MW?%OxO>5KbtC?NH!@YKTDgtDkrWtIIH&Vh_(KySIeICS3mVQ@gk_0+)t zD{d4=ER*sz7nmeKLtAbuaZt=!ysR{aj)=o7hac%3_(MVp{qH4Is$x72t>HviES|Zy zA)`H4TgjM)sF1WQc=64N=8~9-$By>9Zo<^9AI2ltqC2hQ8#oAK%+}4Zr@t~Sr(@&NeNu0-NO6?66r%M7xPKh&r zAf1t9?jZy57cPXd)rIM4wn7bp>qoHXc;}~Z?OyN1L3GREnQVyh3tOZ{{=R0r^g6zB z)%>B~fMH7ReTqvMcy&i*bB$sBoE6<|v-*nm%s#L9c`t;+cb@VpuU*2HZ!}_8#ISzA z7eid#e~PViJ_&moh$<^!L%q9oI{hU$BFA6eK5iLP5nQlqjccPMsEe#WibEq@*6XF3 zCo;qI$PT5VLHslX+p8CnfF<2zWpBo+wrLS$3`js|UG^6lRss)bi= zxWTM>0{ge=(lOxQx31*2cdaNwf?4w*=?Vh9jS45kh zv)MxmZ`8_nlQ@-1qD4&OaW_^T!+FF@{VOYlx)xc172B4j{q{}CEX?ece#4dFIGg?v_95BYFhRLbb5zukuN z#miE)pJaxYGNViv0%UQueok{29C-xJtKtx7xgCOS{hU(rKjgdIt|oLHnkqbwV3{dz zAG`A#qS#kC{UK}{xM}_5;K9BkMn7+;2%y&wOPeaJ>D>Wi3)gH;mH$u~=_zrkSl{ad zpUySjgN)W6u*up`Co53F979x{(+55VyMAalt*Dg1fDG>z2=ODmPTHV&_-4l=8p4dd zH-tcnNl+GPLsG5b7SO|p4@CAfhyr?9?mX`3XCGiDh}ZoICMdb}z6QyT8%Z=uxZ7x&d7pOyOqcf0ms7FD}xbOa6pG|*>G>aw# zi`BX-u(fM{EeKSH`(P^%&O4x|K@@n`ulg}6d*P-BFS%@riOuCy7hSX2=LtgPaZ&AM zCDHK9un>METm5P$OWtxcf}fs+^oR@gHTnI3h)_u3h#KN_bBAXO8uA%qrBz_MUg)(e zc3a_ZPx58kOSS0oi@a@$V9DA~q0@^*$W3f<;Hswt=G}U+4*HuCvM?>YR#pr4n7g+2 z1R)+iTf+zHFZO;)gs2AedUWfT%#mbe!@2M+O@^I8Yg{>9&%4_bf89^crQIw`MEVAu zXuBVxs0r<45SeeZifZl0WneCzT9FCw0ekph_WDLMY54rBNMHrLUH-IK*N>BQbAkTG zQP!2wv@vUcie8I}O2$LCtQ<@Amdp}eH7CAgHo3>YbX?8frWt0h{5>ysp%;D?|Mih6 zlN#AeRlX%*mX@E_Cz!Wn-+T~ay+*AH$z@)JXALP8Wg`h&+30qJ|7h6CRtUVM-wJ#2 z1cj$^Irm@>(bOu9f(bkNi zOK^|l%G_5<#fOT#DcL7J2ysO~3~;$Lb5i`asDnd^pD#8FM311Kto^CJ!_jx?jI|M` zS(lzw>3E-(^S0;SOgMaI<>~GG#rD_F-j~_Q>}bulc-mL{_$HlTdH)<5zQq`fXs2^J zS^b`o9{@>PVUG1g=7O>Wb-hh*LEhKQ_Af9p1abuK1^I zD~Pel^Z=V?8k>g(j<&owZE{X9l|3W7*XT7%mQHubI8)2$waD8cZ*m~x2CJ{t0yG-? zt%KfnJFNjXjqYD%cD0UFOq5~x=y8eBCrghylX+ z5m@jaBtTnR$(bQQdZ=l7HYHej+`jO?TI5a3e-8MNv1er@YLl3h~$oLwmti zbaD6$M#84{JFKqJ4j-^ooPH2o>zL9A|8H}s?L2}P0mrGL6oS`7&biiEqdJ{ZArjUT z`Y{Nv$Xfq;pdsE1T(+Vy;f(EJ{}x}HsBlPCNVC<(gK1GBwnTC|&dt8GHr;@m;NJ{{ z*sn&7hol!Sa4V?ift=e7e+mKhmlCBwu?itK!gL-TWQdS8Q7wjau!qd2KC6PRa22oc zvKHp^d?voaist_dEQP#TwVjPiS9z1NExeuLNDBEYFK>yn-Jkq(b~5@7Rr8Q*MHzCo zrx19@_wMi%=G0{fzSiD6QtGgWVo=U&e8bSngIysgFBqW`Au|pj@?Q^vf&}m=<>WC6r6XRE z%6ri@yWbSWC0CM~fA7o+<5vqyBDj<<1yjQdKnqc_gg)4p;Cwzr!E1#>>p9yLrG`HC zK-NFJVVf0mxi@UdyG#Z85~Z0S&KYn5QxN-~j;C?0dBZSZpN11L zbjT~XS_ge!Vvl#k{UhMF0X+6R&TNgqtMeg)If;C%K;J0;R-<*C60sNFWTXV#vmC-4`7XB&+wa2|4`3r5Pyds( z*!Q9&iWHBguhPK%A|Pbw8r!uVyK{gziSu%qhpUNj-GhgmvZLEQ))88qiuWLj1FiZ! z8c1Kj{7vM!VQ1sW)Q-X>+)jZ5!ZZdgeLI_fp15*y^$<%^T~R!`D(pumY4brFM$K2s zY7J&jz7t}x`86#hxF*6I}!%YVB-2nbm}_o`Do2?L@6nGkUeD?yl|Fa8K+g(db65~n-jqw;aBo#p6n zG%vjDNm?l0fNz`N_J7X?J0%M@C0cTn04+Fhgwp#mABMp21y~_wK(9N68xx>=m5DSU zy;}>gKrnfa+jy*wX;kT{Ty%I%ZZQ^pEd#@yKIrM6D~6qVH^>wWz|8F1CcO5GCB zgWHz>KKk7S%OX^vU<9OH=Ur@yEf0h=ct1t|t=Zqy~LJLFgA8^2#!Z zP*LII0N`qES$$Z;3Lx-4LD(=b%e}mo>wHaa-G&m77elfsD%7#-RA0$&;$_(=!(}xk zoFO#ZsnS1a4^jLhxeB#6l-cX^w7)%O(R*Ex_3>gtx}Am=TcnwZ%5@U9fjs=;O}eP_YvgZ@f*ct@lR==ld~bivK&{5o=0HkkS@*^TuFKNzaaF$ei1%ThUareQs zYx3^x8^)w%0h?3umd#&0q8uJk!OnOwM{nwzuJ<3G)@KFvtntHVnV+D^?|!MO`_E*x zV<#z@e01n+Ru+?kPVcV+96EfkWMd}N^|DH!WkpC@JwI+RdZ3jLrtGk@$HVoS+v zCP30L_lQg6)hkXnZ5>zWj_sHD%#i!6Ie<~ONp*)wO9kXHg}Zl_cW^yh=&_45NUyepE0G)p;cVuAjPlV>t1FfFj?;=$?891+U&u9Z;LX z!=cQ2j*Wn~&!lHlqI>Q1;GC~N?kOX8viR#hyCX8;uDGASp%}~|0V@U)3b4s4 zHRrm|4{G1vrI7U)+BOC8v?Z}y|5=P$aG|f{UqOEl{}ZEk8s#sQl3aJb`w=Nhi_S0d z1IWp}c*^8Ia4sFj_7Ml*`k4Hc_Tz`(i>GpdTb^TUTW$plU&XXhLnedM^$!yCE5|s% z3@LrW4Mq6Un+YoK!pU9IjQI>!2dt%L-Z}F^Z{Ek|CCPa{|I+>AEnNGdclDe7`jsD1 zx5FUp%VdqN%>OuUhu_V>xeUpS(IiBcPRtwpVHtC=>C%0tz7(PLB+D#RI>GOaD-HHN zfAJ=TRneaXvzOE1kKe4LsbA~qZEVz+?lb57nglo|K9oyd$0G(lL~~gRB-fsuB@6<$ zc2&0Qk}98S)uYdB>*DRwY90$`_h0R7Zu5D8RHV*WOJfe25AlCamb5*M2h}WfV%^_p zFsSOLrxm=BYH|!Hz6IcZB%6L~iSp=~)nv;PrJIwuv%sgaRNloARmLQm3FbnjKqqk9 zpmvYBgYvRbNu2S{^B|X(Wo4O0=q~~CH18&UdpSg7tNzxy1q6O%30Hh;v*ZAYhzvGv z9DKd3yPEZQHF-75W1^FPI@d~`f2Je)=l=aS_g?>}t`$bc=g8U8N|MgGdSpfs^l3q) z#(+*+W%t~my6=&hroCg|q-^UCny(!cADZGhCI`*m%yvHXy(~{#80$fDFKNad#I~2# zE4_xKh@A4xYUw0ZMr^|#e!|3ytE---*}U=JtZC1>4jdi>G)?qw{N z;^A(c>M}!I>Z5V*M)ODQ&%PTYs`w;RmjG(XSP3}NbUos;pV zG-03#j{FAfB>&$2gpT8>&mMn6U-k4#VArttOc}X)Pe`i3_iB28FvZIk^dr4%bdfws zqC=6?kA9I8C_EW6Yp;C@*Mg@gS==aeRFby&9w!NVT{S(GmcGGJ`upLZtF6rdo|&BF zH6eHKv6!?=c8sV<%<*1&_U6MG225|Q8iZ%2Pf`AvhsUR6%S1<+KS4Lu=%~IL)lTE~ zU3S*eKU)@)`G344DW%WV`5Jp+b7pKY+g1;4dSKv5e8O5DQ)8{Q;s-2uc8C{>(+I57 zVmb>{`*vn~kJ0kvRm?->xoe48v(%^ehHuOCO&VXr8xz)y8T4TRWNSz2sfvgCx*r!E zO-qiHi&-lSSjf=rU)SeYx91PFEOTu7rh&$!75aJ(t<_Vq?)__vGJ_{nK00XDgA7?y zS@_o?=_($?1EVFGF#6#31rY_}9NhE|2mcV0QA4+GS#hfI>+5$f2492?qXHsp#Z)bRaPofi&TFZ_Sohyude6xc z_Ow}Ci>r#nh*u`3#)=-6@`T)J`8hrMTR@8mMyi zc)#S-RjfaeL2zp>M4OlbbR54C&H1En@v#z|nB_(KYTo^{|Jo9oJ~3cYdU)$ax5?jS zCb%-n=;g?pLg;4 zuRhEvi<01AdgA-y?1h`j7bRE1?*(})(3TDhuh8Xuw(gJj#`F!^mZd?xGGS^J&lo7k zJ1zsst;hMiG~~4!SjLWT-+nWKd^O)nX?Ph)R*riG+hcuPy={U^8ITt4m%nH7USE1A zyE}G5=hnp{+rCO)d`_hCSj~9_P@65{l~RR)Zq^S;Y)T17lmfd?P+iES;faAi-ry( z3iqB{DAyI|a5BVr<;;kxIa&3U-`*IW`}>L5!@t2zREDZAGh{=+k9Zt;k8yQ*S$OB&%7>@ z%pPI;M)+s3d3HNyYjl1p0^&CiHLfM5`fU1e#iw^)^_?-DUtHB=?=rtHR5$V8RH(){<4x{wyz@Vn z-~Dh(u~#Dan#D;KdLD4O63lFC(m)E5!?&Al1qk^Hz7FQ&C1yXO*XFDf{r4+cQ(^O8 zjC*UXWGAB6_2wI41(?_C0-$I5mO{43Ee|^rGFuAa~ev$<6tZ93WbM)qU8&>x<38< zBQ*PN^lz*C++WwfOH3=-jyt&l?Zqq?hY*K`kYT>g^=qS^mk9zqM-8>~GPw$8WFL;M z$g^*VWQM9OM?{6e_)|=Lmyb;oD9cijtIkr9I4&NamCJ;Fj=MK$Hx7|)@|pvSZyah8 zso+%t;ncCO_WsWNlF9td*;M*jG3Dk5Q~&ym&`YQ>`K}p~eypZ-p1!}rE&Rg^kc`tq zU4chZ=8AP6s%U<&!Hga^P(oxIJegG{zD~$33f~zty#3$Q???MhW#Tl^KljQrD8E#R zFfNTJI59sp{xALwf2Elii8m^0(NAyglDFmcv45GwE-CryN$&5LdCz`uCtLXe!KEC^ zd^*ibc`C+&$;$~oCCRXyE{QuLC8to(^RuyTQo{e3id+ZAdpBN%R=t6i1_c0P#AW_e zVX6+lf9k223@>;8O=Hgy+vky4MDgQQnzLW^eo^JinWSuO6B~;1ujNIVf+8ROT)Jgt ztkE{?2}((PKDiP)POrK_rTM12f>GH(hwEp4Y(TQh%euUpZRK72Y_jjD8g44BvC8Uq z)jMyC=c<8@Vgl;WPhVZ-C}UsS!$zJi{CfJNV?WfE)_VUzRW&%JLB?g_eZ^h z7^}sHWB&xbKU&}M|FqZqvobH{C9UD-Z+%AgK$DdRj;vK$G|xQa6b_^iGEyEANwV!_ zf=f+5x5J7Tm;Vd(RxPP~kGxk(rTZ{pyy1J1u`oQ}D%)--blgO?QB|+rzV+#6Wpd%pHk|1TXq)*Xkh5KIsNIDsM_)c<@D!MSyUKfPfmrI+AMkaEf2hgW_UfWr*JzQigr==S7LJovr zx#2Lgp8~EWah}Vq>}-z%L3oqVW~mA;46Hsla~P&#$fcfCw9BGo)W*_85~mqVniLtMwru#YOa$P6LvssH$!O^PT)yVKki#tVfdD!Jab+QhJCJ&3tg*e!Q zR|r)%WQzqpQM{K(nC6OblC2g?P@9%VMP8OYNloo!wD@^p2U#yY-4}oW2}n^ba8#AD za$$#^WnqgKvS^fwiLzy4+a*5cDp+gQw~1i67O-PkNvUOusO%RQYSULWlB<9K2uZXl zuIOH{QsG=42y}}>B5_4s87fqHX52aSV$NhHQ-RBK+C@!|YfdA}4fp1Mm6 zt(3c#iAnJlBG&7f}q?o;r zvJA_s41rSll~KLw!mZ1LZzZ0ujbB+guz^(Q$qiAOdBx0nUn{~2eQSy(Ns6Y}Vzw(F zp|!KqTq6{Ap!)c7**E2;Fmmy9h-J1bkwQgmx*vIVW8mKgI}8i-NqIxB{$ zg|ZN{mJFfFiQxvp%_%WYlVWtv4ppmJ$xK7uw-wcNDOkNLS(@}JZdM>^Jf@A?N#3+O zF4rZ{s$EKy?ZYOe9>`6XCem(6UsSfRJ{HHVRiW)EUQSbKM#!a+=Xe@2P0m*!=3@&o z%=T*!Oi>K8a9ZBXb7ZO3(wrSDwpVo^S*i?M45!+3N{)$@GcCkuDCmiegWm3H*w>*n zEK*cfP(k2Zv5r}p7Tu3LQ4|@AY7$s7r$}U~t0UTpYGGHxG)vWn!bgqE6|uHkCiN+g zqt$u7ZoyovD91%@2H}%cD2ZKkao~j>rDE>u%(A@#SAtF~5{Rh)2d_vrS)c0o#mWpYx|dXx9>5r{k8MBWlNzQZ_2C;}T9bVx zDE#943d@CFz{x8Bj)qZpe}#Dj=mBui5F{R}13tf6u!0f^)hD{(0oiTEtYElN)ztw>v+v?egm(M`vDNBjA9Jb zEA_v5bWLyl+q;nGq0p5#K^>kB>YtZTnzHKF~raZIK{0y8g`-JRf~!%?XEnKJ zBXGIL9B5RqNr6{<+;%xXAn4j4SAfO%2qf zssehEnh*NZX}{ir1oWRh)w$CM(C?e3z=}@%RMQE^$)16sAh7`^>etEn>Y~8|KOvUE z%yuww4{#<}{L%|!y2$BbFlSXJ4-JQVerNz{1DjLF1^w zZ?K-YwNS-gAuDWRM{r#a*9Pl=-jrP;UOi>^I^cj1c@X^g$tKsTruO-4T;i}1zfenPg5dy-eI{e;nhCdJ!3j&596+iikGi@-MD3v`GEf!|y$vIXovytV0a9TH?A zjTg!asdl-<1Us`@El7>v4(aed-}@fu4YyI#$Caj01t^dajLD1Oz`fg$-K6BbD+Jj* zw^w>)OuJfmO`7BfIoR#q3!h9VyvZ#Fd1zq*%5s5a`u%m$Gs+kw>ufk@@|-A&7^sQ52-JVOLsc2=i~(TYL$mxn6- z(Gf!*-Si-IBH_b;Js>hnacW{1+}_|vZg(8_Ff=J*Fm%z0)q07U4sR%h(XWA#kO#Mk zpB*GLi!VKl>|V;%?R-X?l#;pokc{W>z7I81sU5)N)PulYk`2fS(>aGpn5kp^r7$Vy z#Rsvt{IIlO6~5yB!%hI_541)N%#1iO`2d4E%skM;EE~_Vx^mHYDWQ8^Z?I^IfJLv; zWIZK?-ysA`qDud4m~9V)61n-wxt$}bq)t|0)jU&KIMtE$1h9n*xsYelf0J{gJ*NBb*M8;&GnOJ%TgNBK!qb62*Q2 zZ20+z9b@%v@S}ux__d1~7;m_eMs!S-^_4-0aW2WLjVw4hX~*lcfnN(%h>ax79RxuwH5Y3UOS)JsDPc(i)=g2BMXQJ z2vwi|IG74ZxhKaZHQ*Q6&hWFQH6+4>iOQKs504Q`ntda zl^pwh&mgGScoIrUvL;d9rc^P$+&1ry4at`I5IwmBP}S-M0>$dzl&gKz?sbCyYC^xnMkfnh+6(6-Ukx<2*VL}i$v01mm<2nV_&g_deL=p9(5bHw>Z3v>3iBLB zkm^MqQ^9ma+U{yU&QTMBQ(j?Y)R~lT6Y0ZC&0ka)g~MA7$&m*EZv8l02`Z5#i?JV= zKUCWdeHvtak+dFytW~^KaS-?Sh(`*M)3(bONZTtR2&Zn)wIf5$zMt=yAz35rhWxC@ zD6pZ@cfPNZ(wR95Ur!_Yf@RB?LOD5S{Une5=8N$6>M|j-0JJDm|1KDMW!MAz8)a@+K845V^}Y4y<~rl+ z6+KXX*7~GUuy+AWo&1aoZD$=SUngH^Hc{#K^yxq4eZ(L1_Flq^C0-ZUuyFzoh% zZU#Dt0NJE<=*YJK9aX!!R1N6(V zEjnZ^CfPQ2yu{?9Nl-DY2`-q7QFX`Yfoa#urpn>j6~P3rh-11HK6Wq@c&8C(d*BgfmX{KcTUskMV?5b9Tu3q zP~MZ;Na6wSZ3j;rvr^x0iTO-b%J7Xmn?v%u#FGH?;hxEQt-{3G^TqVL zMYFa)<%qmOZm$`@S_2b&vF{E zg=d{Q@MEUWGJoUvf)$N_t8_>;1;-sYr#9vYKUiIsj7JzhFJ)xn6zo%~N-n~A{*zyV zi_ltFJ6_)(-=T_x$YSziAAeYm%Tova0e;iW4>g-F1jR_^WQ^y*0q^CQD!Dquvd|5B>-!!b{C#bk(rC+!nk5j`pu- z?Es+_BD|c!yRK1nz7-rV-MZp+qy+TO7;i0q+kE{n(vpG5bnl3<$9s{)k6^=QHFj4? z0~WG{dm8Y8Kk631o5p5q#6PIw6aO7UPz?4L|4yfe7vbLv+Omhf;K08kIPL`8!R^wr zybde-x9Vtc!)=#NG!TEsS{>aTQ~&}#f7kN~NW( zW&(Z3<>{mHDZdiAj|6`Erz35BT^XGqfyeJeko)BStn;}`YuuRx4F5uo75J=NA34F$ zxx%f#{J;fD_CufOvQEu@B{b<$6`1vO8XM(CSgKP&C6u1egd_J6${3E_c-3xmoDM4O z7~zjW&Xebs<3FtKcy0#j)reZQUw#0+jl`s|ic0cpW0TmuuU@y$U=Pjn3{n7T zK^%z*`J8_pPFPqHXX&!0JQH#~E3@jbQ`6Q%U$YlueT+MDO$AY8PVsK$4wlz`qyY&a zO_Ci%3u@_U)&zBI3sc*Ue^dy-&IM1yTbA&J&)+v6;OuF|m7;dQ4QeA>eP&V2Wy z6M5PP1=M?b;~RWfG+cFd*9CdHRH!B}-^WJ1jF4wSPEm8`;Mz)-eAU-`%t)%yE1vFv zye^hU4@Vs+A%TfVhAuY>^0A^MQ%+wi{$i3p7xvQWQJMzYh@MmClOlP8^DR1StAxhw zA;3+-t`9^0KoAa23`)8Wu+{X4<}=l$4cW!uMrj4T7MNXQ>nNjz>9ll<0#% zq%M+~vXb{!6S!Y4uuAIEzfODQr3IA|?>Y%TlLm&ux+NvGBbjA=S@MnHUtIl3Za%sQ zVZ_E$PljL+-s0q$fXB)->=(GJ@#T50M~W=5-(u^QJUql*M*8m{p@lNb9CmuAL8%f-6P{59 zE}&8MHZUZh8OtJ~_(Wv}pD1yt%<>dinW4-u#Ve#NAn?_iv!=@$VUtnW%x)-$C@G*G zrBwPf5DbAO7JZ_yCl!RGkF%6#{AFG~K8OS@$#dM8-HO@+dQ4Q9lop{@Y-& z2u%X~`jpZ5MKy^-JUopm(9pCasl87vK=4^a35&Qd)5qhTjPg1x%cwrE6fHSqk_@>c|vPtbw3}^@t zbu8+APxpgf%6V*D1GRFiB#fn&I0a2F11Pl--XwT( zX{@lw`#{B z=^FYTWVqe^A39bl(|v{*ae<=qgw~;2^kiBHtxLHra?^ky?+@+0`{yb~lCTr){n~uf zK{ao|5^_4k;`Ig|or22ESUT@tVv^olTB2JIDmzu%_==;^nzvL+*(U=QUD(j7rV5>} zkXcTVd42KWli-N1pnkm3Q;?v#Kiab9h>g08;oCESI2Yu_OSvnoju_o1b-`2&G2Qn? zmmowKz#jk&B1?f70!#g2;Emt=q?tKSf9z%s48HgqSE34HyEzzaN3&vZCBx54TJ2JP zhgScp$@9YyJdr`=`-6~7-;t4V{W0p= zbY(Z0O_=AC_n9%SKHiiDgif57o-I{Cjy>!JG3Mwr76IabkQmxba;LN57~Kwie)w#U zUJm1mG>J4#nl*pk)!P|ZcgEfI+H5i*UBe6a7>~INXskl(in&Zk$84B@EsJ8&nTepB zOCO#tq7+OayCv%0?=!PvpSFND?%qIq{)qd@SK)_QN!Mw^jag5z=7c-z;yawnKz=DP9vIkBIKCm; zy4%36T&_9%My3#j^)YbGuKid47F*0@ItSOYk16=bh(VdR_IR3%qU8jpFPaM>a>POy zFAKY21I=9OBh1H|vCWee)A0d#7n${>GTy56K^%|WDOeZoS^>MK)fe`gd{!@&-MIwk zKz0O+DSw|>EWB2NAYy?;D8KM`MOZ7T4$o#2hoJ2}uP9OHtXY zgm3kZs?B1rK})BrC8aw*-984(dn4o6BnoQ6#}L)5^jmOZl|hnyjBFve4rzIjlu+S< zRlmIVIZ|XgZQA(zFX2|wu-XNUj6K4FP<9jk=swk*?l+FjiM2Q7G`c{XSeKPNm{`}+ zM^>GBd1_`;v2GJFliNu4oYdS~JGx_p`h60ek6>@@HIH!;iu!983J&r6)oR5J!+n=ir@8{ZELF{AQcD8LV%f02zAoOR4BB)ntL- zY@USCH2pIa#NhW|l6foV*pA)|W$LsYSY}M0XPgAvL3xNmrK$+YN2J9x96i~_5Hfn%X&YWFh*h~Yd5VFlQ&T7+b z@x<8{kd2+!w@m=7m{cDQegesLH~fUg;CI(fuv%))IYnQ>kbA{3)w_M&*}}PX))we} zV1~$+_*-2?SZ)dzZup%vb(H!bd~AfD>nf2rb@TKW47Ud?ApXwqJb)(`6Q=9!_2VZ2 z*5dykHV80(w+KJkCZe32h`0g^?k-oLgGhYk&y;K4Emxvc-`{(b!(-j0n%`Imj^YUlaON{ z5}bxS=se?RDC$G_pSK^e@cR1p1SjzKo2E9NvYvq&5&s*@X zhlKjz74>d1tgP^af$kIvVBd|`s_l0ls0qqj@SKXeAu(#Q?Z%j-VctRS8Ua=s{?13> zM39e$(#%tW%;ovX*&FeR7P89$_A)*{eE{QU%$+J4@GC#UY-6FtWgH)WbV$URuaDH} zom0n@ug&13K=NMBFyWiYECD0Hj}d9;Q5%mKtPia`|HOnw0jkYCM=7el7XeATn3Sf1 z3rIK0a4euA+d*u{V}RA=#7nCPf-I0Koi zeI#+18|>ZP!IJ@pMKX6GtK?_aQ#JWMG~s9hIZqW;fSlijIe&W~&~P2rT>0^0_SYj8b3V+v&ZdnxV?J2z_lGBkWhwCR*fLHC~>4VM4UZip34A@8uzGUkfav zH8SPj$WP9=D~P$EwXw9^!PJIT40mi4<3|QM{r;5)G=h#0FsnWkvw`WIvAJ)o!ue6F z{;=l{LGej5z_@Llk1%~x_T9y&uobT&qB0Fr{fe*gPE%=%i;iiTOFIIACUZmspRLx2 z#Wd6ixM*zz72Cz7&~~p}NQDcf!ifWghxrF&IRG7+$+EIK7)4(xeKMu*d+CP2K?DRv z1mW#3{~cL5O^L~uz1ONzDp4s8cfN_EAMi6#)4whNqn1rb{s}wICBX1VB@{Nx;Tygs zVbJnQ^c7|uqGi;G<5tL>EMfBIulR%VJ+=M2gr7^q+a6Bf?kW%!M1sv^0KN+wjaahEHEg9JC%OTbke>k7m72cR)mTa0hh@@*fxbw756LmkNvp>Xb9 zcV(yF45M1cmgy+UMC4*{L+M-<^_>&G;h~zER?!g3hi&A42)s65(QTkRAcX|mD^v)^ ztE@QB8Qal_2^*>#;q$5lNb25T?h^3FzRfK6M2RW)=j>tDpZiv+N*to<0pD%fsh@#U z5T_JBm7B4@B=7ay7f860R(O?k1lh)|HoBIsDl0{rhm+k5gAnhxQoJkN=?Exwt&lr) z*%P~LRQ{NmaAx+Le|TvaCRDm(;y5gF_qo^Bqznp{!xK6tLGXpqT`0q&kCi3!1rI0s zI6swBErbTj=Q+0ndwWi;tUXBY%Hg`Rap+p?cezNstc?o5(e)tJvW8A@T-338!plEd{eWW5_NN<(Piq<&{h0Y zwiB)ajfea#$n$GxM1u~xuj*Zd8UoWmK7}UUSmZL$C{Xwx#V%#nkD$ss#5~FPd>Lea z_gK-i4_`ZNC(lLU%y4SrTKugUZUt9CH8Y{bi)gM0mfcjf08zK-5lu7pQnk>qp4eNM zma$$Xv7?z>C+Zb01$M~wfEFx!3m(_9OTW4|!f@UmE8Vqfw}9&l)L;pwzSejgYMt-W z;{l*grxf;%k~@0T+-GgTV%NgI;ie5GQJLqu#L07A!ckf zO9XTs2U^N_58wYMTM|LtKO{wFbR1%dGNBPx> zjeTUT8L{`^DS9C5jO|<(N)og|2^~Sh^m*5f=+5Tr^CY^9V4xwXR(_{18r}D1Km4Jx zg1&jmxybbZmsp(uMZ;Z5a31M_9v3-dBkNN#q|IOHC|o@B_+yn1D>$w`7g)m->$!4e z(^Rae(n~g%`}2Gnw;$LeO2}sGf6_y%`Rl~#qt@wk>VGLjPyO#WJ>KXok?*Vi07plr zq2WFml+Wnu19y90&`F;tcyPmst~`y+o1eqp7R$=?&$9-}D-!yl?YRL$-&}7j_J{$Z zznid#0j29WIkxqbo7#Qg@L@w>;EHL$dp-?PSQ!Po@S8tu&`t+*I}GM=MA>W@+_F~+ zLV-$ZkQL76RY_OmYDT)8>EWDU}Wrs-?Axy_b zrksD8h`imgf{wrjgF~~vu8xT(-E4e>!P;~}w67nj&8A*s3O|MV;{dXI z7JLORtHQjec>P0oww*V{4&UxJGEV?Scf61C0w7`=VaT=3D&_d0mMe4pXn zTzt>5y=&kPwZOno%})~5QeGgPW0-OCkXyPn9mf;1K-Cb~%=(&1qd7gc_t%169=+IU z!MbJk77DYiB#~O8u@Fi(%&%rtLQeR*rPg&teio)0xFFubKkjSJg?2JA5PNVhwJ6{M zrWZtwz+wcA82)}(eRqQ2eR7SG21|>TPZnfvlhNK{H3UypSo+su=S?H+^I2TDWbyDw zzISS@5Xm9ndSFQfYzA4pktKc6pNXPM+3AIgmh>B7leZ&A0K(@2Z86=^pJfnB zAC@hmYsf5Q*)RTU44izPq2-zaxt0&!H-dAH2e9`yA!zBOkXhvn&Tkf6@ZA_)Y(nn^ zfHkNd$Ux24YR|rR^sUX{v_>Sx6dX?q^c1HfdA;pYmaM7l{ScPGe#!U9t^cg~1e*;C z*WiwU&{MsV0@!dI{680>e_jKEHhVX&@}uf<=4ED+30pQxn1D93Lm4G1OB*24YdKW= z_6M=vD=XR zD;oRWKY&B8vsP7NTW7ZL@o~YwGnE)XP?6SlWm4Mis|axu;g0JMIKFk;W76@);Mi?{ zH_h-+xJq%Q*0b7c}@D^e|C+Vd}mmR+35X5VY4a|+(o zi&)_wjqHKz?6uhUr+l3?gWwN9GH|pU+$zk#?gVS@BTO4Ixb_79@B;@&q}iXiP5E-y z?2Q@=wtUY2R7|gsyx7CHcOwh>H={m&bdr7i{W&~*l1wMUfn7AgjcbSYaS$&KJ?3^u zYrxg;=Q_Bbv>Yc~jr)h?#%jlx#UV(P;09u}^(=A7Q*H$V4#Qxtvb)1<=vp;(_@|LEhlt32HE99L5MhLv^yxx}f;!3CZ(In2%FgRV2KO3Wl3vdPy zf4}s@YCS@}5ji8*)?FpLH=uq<=f{~}+wl>k$Gk-?a{?$V`DG@i)d1)7S|8t!vrflc zF!fd=!jtdU*%0z^+c-!2o4<+;rswz#T}Ur-p(CZ@D>rI}fQ!PFxabpFaS9jjX&+=VjyGd%5@h{Nf8}?UHx)3I~(2)LVh9N?Qqtyc#C zb8}RdfwNji*$y`TDyp`+#niY$jV=;z-KP2q-OwKL&;9G=DjnSx?swB|tFIfwr+ZJ& zs`6swzM%)xtfSrS@8qz5Ztfvt<8zp2+fMM_j(az`M`$IR_`6Tu^JDrS4_zDnW*$b) z+rtD|PeOZy=nB}o?x}3I)Wcab;g^SpK^;hAhxi;XM0$p)n;sr9zw^4@qh6C#FqAIy z7-igJ_oYZ_t9Zh+|H3Vs40^k`3lGb%I-MuV8=}w?EhgFNUqDDLNq{@u?(?4~0|XKp z+`f%?1-O^N#e%M<^2+mJ1y-KGs~8e`9f-}ffqdmM5l=KZo>S4*S32inK$EzpwYQ$< zcW}6-(hL4{N&&nZUi{L0^>)e6%eYij@os1^vAuttM}oZaYP~%~-XAzmmCc0=16lv` zkkNR5=y`%q3j z>21a;Bp7K<8ob|w+jx7c95Z^)t-@)p{P~=Xul?hlkznE%@jjgk?_6I(|K;Sh11a(^ z;$yY*MFjgW;PboR6)Wc4$AH8h`tvVd?EkcxJkIk1GoLhKOb0)Zd zy>eeOD&PyxMwc{W&yns=w)xIwVJu4mzQd_LbKfWW*u9xi9PqyL+CBWl)epZ?144`A z)8oSbBW(yFLq7;E8gI6R_;;8TWa!YH&*#k_hS@`|tdokx8_poqkxO;|S#*IyzWs>U za{>Mf%b@f>m|c+gMN?fnH)8z-gHXDM|GV^l^c0;qOE)Qx%X>qU1}H_%h3p53zGZ~Z z`9d<+uy8;+3JdpyOk3Af;{!B19# z&;W-2+vPYl{i>EZLNdYN6ML7wWE@8rzP&3@zTtkMVGdIC zfES{Z2{yNLSJMR! zoe1lZ4nOxqIU|=b>YzYpb9Bw)AB&=pj4wB#A@`HtLhVWr0T^IW{JNhHmUj8y6eWL| zzf?PH@4J&ezVwBC7Ttu24T$jvz|47iIJ`yB@bd}w=#OdO*uvey^9`H*K?IYZ*d zjIT;nmX8Im(I~Mhzs6|)i`jA&YW@8zk;f`JN5p8dF6II?8QJ~T3h&lOdiD7k3z>yp zs=S51{bHeOhXS@sg6mv~wejl&9=NxD498eo_(F^i0_J|h?88KC&`Wsk(@Ug7@gfbp ze0GZsfvkyKU=W*<({T7}_>c7dZ!$zDj2n;bRbP8t3A*lve7}h}aGxFcd~b0CeoW=S z{adH?FGP1Qjydg1YlMD}0HA5b$x202##tr4ZO|xwT%?(hfFX(Yej8c1?b}?&?KOvt z#l<(maXU%O5|lhg30wNIzGxjkf;oe*N1_g*?5|(c zMJ6R(SDrrChU1RBIc3 zzV06cQ^E|8pklK~GJo!}o{;>HYH2@nJ~qxIRvG2ZX%|GVi*K(3CpLHco^R3|L< z#)Mad=D~|60-zfc^!e8&!sJUL(;GUTkmHs0P<&XTktBlN$Y%<2E-&5r489jBYeZs_q3I_cdQyW6WOSVq5cZ5n9rYUtC-2 zgRdV}h~yKNP`F%oIx->ZWdlq{@Ykat(zOo=>kP8 z`FESRc%CMe_c!-bA82BjURHSXtJJ#&;SR>hjNa&oAIs;~TSDqz@5H?gngMU0?27?s zKN2N{ex2a!_Jwj&iUt4Lr~KhBgd7VG`fckf9=7FWM+{826}~H5nCYyMmg>UgD`%c5xT*z{NV$|};38m`!0VqBdmaNP`I2&+QUb)6-P}JDiq&q< zAZ2fVl|c)@5vbHQVs53}`0KBBUg6n0#b zWGADqOAF)!ISa$D6KoUDn6)SWDv2=O23JS+Zk`G$%)DQE*Z3-sL53R2wiNAYm~&qtc(p4Ax=$9atK%y8-r z6i*+v=N)`E;Q?6wxNX|_`O@g~zNVQRU;Qbz9OONyRzQHO&>4dI@rXs0g;_o>vdgZK zAh>?P})e=j>Iq0HA34mWpDFL2gHiC;DbDRI-So%wYv!@X(`mz8*R|LF9k@5iSv z$u$rfI>*No^$#j0O2hkEEhQo7iB3?`&)+a?R zeA`#pYl4ue_xQ` zKv%W?jXwNukfc&6S>i$65%>T_Df#uQ)VM`$NAPb>OKpwb?mphZ-V{Urh+d>L*!H%s zbdk&#>;%1){_6jhpE$XCJyNvOH{;xV97_m(esE%^yb+8af42~o#gd0cQ~q5H)r6n6 zY?Cql$@CjMewhYl`@Y)ys~j$njKzZv<+CAqxlTM$pMQ0<#2>W86DlrI;T_Kv zUPsRCFHGEuGfI9JE`IMKODZ7_q&zE;5Oj+K-WY+ASBWd@V1z2IOJnVdvVoAw6yJf| z2l{=nOO0vDUaxun@J65*eKRk?xPZR?YxZ;xFl7xDQss*0_-jl?g7zkO%@x2k!=OMPJ? zEnxVnY850kCIs%OEHb`ze;E;$Cw4$(oWkX79ac$9-S&%d128cz~5@BhrCSpih5w4WOZ$ zeKL@;@8EYNb1!>l_oxO2qZMNuqm!K) zqmKyGV8VuNqT{cI5k9Z(F@9Oi_z=}%Gjrx?bh|ygTK2G}%|D0}aMt>6;Dq+}1ITb+ z<+aZU3hTPID4lIL*j9t~wKHIc0-(kyI$-TNYq!zh?@I7}xK(ie)&ZefaOX=`hd3sU zn(YVs*0&oumS(9_cm92%I)5w#sh2~+^8HsA{+~E0z!daQd3=9$F)bgS+fn{;<7+0L zbps&jIi^I%>|OscxBbg_{g?hJLCAs_)=2vL^<5JiEuzrjeM?^Kr@{XKzB2$7na9$q zKoEsrY4Xfgkt8ZX5nTS{oa5Wyxf`pbuxNU^&-qUI$+oD_nKptu=TZIQ?5%{n|23%x zqJF6H-(NH}>wI$=xZ6y)2c4Mmi{V%F@Dig#xm;QnAX4;DU6vSQ>|x<%)9xJ)7E_f} zFlO*=-?y_Q&Lt^9AB(7M7y#B!K+=|`F9vvZV(mJFJieP=JAIun2TJC|_fD(*2hHu% z@EF%&{A4`E!R*IxeT-f4>0^&tqqWBjEb`hu4FC=C2e1i3?Y6}}GggOOFv4*FtkNUb z_;23s?B&G~eK>2W#Zk`4*eB-iR zZ}(PE=Ckq-Q#W2H?W~glL3j1q#LEL=l^!sG@%mx`Y_9iZ!;bdie$qceWF~;Fe?WDs zMtepH_`~Cl@?~yylz{v^Fpn`SE8|-1+jndQK?#JI#(~h8tK6RAQQknD;9S$nnS$VC zn$B66>K|&a2_8C;P~?(?n)D#a$QCWi7UPpJGDImY3-I@wMDB5(aKgleK5)^BnCHI; zC7KGqA6YC7ZY2W0H1^kxCwJyQbB8la6behe*{B7{<2N*vPUqq_Gpryqe0eJ_cx z&fkyt3-K!}1%SU_@*6)~DTN1;*tLHj_ylhOWS^5OSa+Rn4ofrdhs1Uy8E!mqO^Vy^ z4&+R0v655|mm&+#)JuQ(Pm^ZZ?wvLH zY+ows*h#Z)!Kqx2`eas{w(o{rjqdAY5x!|@TjZ+XoRZRyJ*Rj}*Ry)gH0|Zyptn5PzYH&o3--7fOcmZ&bfM4G0c7}We1>w- zk?;I>|J%c_5E;SI6(#IayhTQ1Ijb~n^1Fj;z%vG3ta0I86CY>E7_yv#y<79lE=@j8 zECG&yJls%r{Y;syANpEylqs1wQL#~f+U_1IVS5AT8fU66jEl`)!M)N52?fv8oO|pd zz&p+~t4`u9(}Hlr{S)3ildF$mV|6Anvw9UcS&-D(AQjm+--NI?*Mrw9U-~dh)}DU( zl~AzyCN>CWuE?_07JK5pUtB$OJF84Z3~2r}E1&l6^;=p{W*ss!fx64pZ}0rh9zoXe zTC5xTa5J0otmlObnnw>TF0%daN34W3qz~ZtIJ1K-Svq&`^2jd@EwWR9JO`Kl(1mtt zIk55#!0>&Ee6i-zn8sO~y<4@Q=W_3IBq|qSyn4oy-*st7Ia$){bt`f@uf%DI>#^#{ zIg-g@OY= z_lNcKqp1m!wTki|RVwqM%k3!gKPlpUNMjdYt!&QYUS-$%ugH`B?^QJuF+?7{zInRC zG&=F6`Mo#CR1}ig2)yPt9Xt!* zylF>@UM`;U7WS*xvByK0+jPx$IfS*UeKCD-&ynuD4RHZ9MvUTFJh`}=Jzj$P`znBL z%r_m2DdzwLHopON)gdk*^>^WyNrfzG{{nk-X#vaCI`q9S5ENcS^WR+H@h-oSTS0F4 z4-cA{1tJ`7ydbe2Z`Re(s%izC9<)pg_cHwz zp1)ZfB=kzO;%#r}xP^cQ)nUrl3keaEtV zUQOzJ(y2+^VyErjJGvFS@g-C#17a7aAJP&qlOMSUB>HeqD?tfYFYJp%9fH62bVOuH~V9;L`l2ngkbkP^AL@}zI>wRUk*XZbim|i^#0PgOhTX>w{b)n z{gnc3imRBV0@)`h6ASZ}msj#N%Nw$$_UcF5t6B@gLsO|nfGp^c(57m1H+svPS#MoS zcVv^O$h?l%zag1;i)mH9>E(y9yLQ5c{^-Td^IN_b*Ev`v-rF-S|8B(VDp)3BmwU*{ zYiLjLFP^6G=VUEDvWtWC?V5W;m4l`2biLYU6q#PLPvGQEdTbgsw)J|R#*4td9MuqHLy@P ze*#fZ{pAeeSy}dfD5i?3yO!9U8^jH-EZ4e+fB1IS>zT5531jv35Y>kWE@B)u?2{lP z#ayjEE)x{P)~UYItni&q;YFo`$wsRRz7EUndW5J=iYn~h;0d2NjO{%<&wuap`@6Kxnu9IEN#qJkTEqpFVVJnYp8wXijw#$F8#`WvJ&J{D1ppWm}7 zya~nA4(+}gi}+TJGq^6G`#{DjoT!5@@uQl_IgP(6qJOhOZ6dtJ?%&l)=w4YcgO*&gw56QN)}S1PjuWxjTu72!oB3x^u%w{)vpPI3w&3x%4;TW zPnGUG44azo>hE=q1AaGVeF33C63)d%vErEeQKvlD+=pkx@SDh%sCzqpJc4>I}?UZve%DhHBcMZW-&6p{*v^5w2Zu zCR=I7&=-5%KxmdIsap#Ub=`g!3vzTMijen@KUHRx=-$?y4Jyv-Ij@R)z0$9eDZ|~6 z*nJ_<>D8Nyq?5J~uCqJ6?H(G;YNn&?J$r_GooXx-ZSY={p(zaMVq zsZ2E3b!H2SXoD28v6fxBCKP{eEu0A1__t*?Qy0`S7c)-a@SJb-ZnaDj;otMmNRwj+Jl?7395>UQsgS0|{&@o$U`D;Ig2V_z-^H$2hj%RQIaiO;DC7w%| zsm9-78{U?;kGMig!X6m%HdaM&ZnS+g!Doe?G$t}bjDkCJ?baE1?~+S0pp46oqt6Jc!>)AI=p0!x`KKGm(wb%4NS9{BoP+R_cR3TKJ% zYb-=t+317Yy=_Vk7+I*__Plx{61gxp9s(aI}OVtX=UCJ_Vi4sje=j) z?)WV_6j0c7`0}nKk`&2nyz#jmI{f5+mq-p0-#*u0E_a(^{ODNzzyy;ZAy6{x^81R* z4I|$0j;qsQMxBZ-<1@Ex4z6~V#Iu)o_l6;YuIz6rkA!%i&%kyacg_hud1*Gi`o4(H zC04zlF2Y}5zZ2Y>egEa*?EZo-1%d$HrKr@fo4r@7?uy^&d`Bg4FX`(g-OSfVd@;MC z!tZuD!@N>qSBl|G_1|XgrLexKG zh^vM_u#XAq9ew)I{&76x7mzF7zk<%WU%30}zv8dG#r-!ja4U?+f=$dn2>%Q!`8C4m zdNnS6JhlM>E$!2cyj zA2Uk&MWmF|)ie~lBgB&1r}r}2=FhMwkRjNm*^vVJ-K>Z1Edz&S+o(T}BD3>h8UPT# z0XXH#qvMJ-lL9ipt6$gaO#}qcO0a^PU>Xp};8z#LfNxtH&!fYRXEw+uH=!XcWIj!o zj8*$5_&~1_0DJsF@u}~7F;E8~uO1j&Z3eAdP`DyuMEu3*I5m4FML=+h%Un$JK&*Ai-IYj zG}783U4Z!v432-3g8g*XQRR8=kgwy*m&8jVX}A)`^PzCnCwV?Mg_C9)N=qsQt!~{Q z<6m)zj7i0@vt&nDj+14+|EC}nzK+tas5uCanq-Hw&TdDjS+~(dH{zmd zG(}CG*?9k0Djc6$g9i(xXnTzQPof?sufWU4uj@o(WL_*+AG?#- z_6L)LhuQlHK`zVz5T{-MYbCzoQ$*DOV8N&d-UdgXp=IQPQ!5c~}S0{aX;pQRbsg${O`@$}| z07HF9HMgG(PM)B&g>nOD+SPyBs1h7Z{_~WXsaHcmNmMdde0>l!Q*gT!E2s<&NWAK` zgwzwjQ+RB5Qe>g*`;^356|F$yr;C*-<)W;~M)Z#tOobi1bah<{kqJ(NM>v%|el$R? z_Ee{QmVl;lM}pX!m9{>0B`N9Gfbg_r7)|wnzu?r(mzUh~Ofpspxrwg6n5wDAq9?EB z?F3PLv!YAQU)uuC*a_Q-P0_Sx)2M$W1PS}i_-OHimI7rC(Dp)KL zh9hsmG@nHyjVaG; zI%r^~-%zlbOMJwJBGj2n%{!U=opx{j=qzhyg?U#tC7J9TkKzyN{VX1E*#2_x;h&Yb zTV>s5#qTJBfxfj2K7TeA$1tmqirbsgSwo7K<%^UB&6~$ZHor+Ks>zuHI=saUkfwS) zJ@gI?bJ{aBm%(|P|IBr6Z5t;amY}IAw_6V$x4B>AZJt`{)i&}5M!OYY(902g<0mp# zh@nv(Sks?ji?S8owt%8zMIC33se~)qu*ILL5Mhgrab)YOTM6sbl^ofE;`yx?s&OIo zSULI@J@MGenG_=79Iu~6)~k8ne`C+Ub-h}Q|BPKD>n$7J8F-zh`F)a1Ew8KoeUqrM z8~@cEkh9fDaxlHq)u^unBkP+WzdZq7A!}gF)(=h5`n5sIUAO|8zpzUla#cvUuHi+e zuFhHperu?8{q~SnEr0r>S8rwCLHC`Ctno=bOpp8XzNT18)LNmv)Bu6Hajd;Atmul{ zgEHvC8(2sB?jIN4;ZJng02BiL9bOh2zu2z*=<&SpW&`W{2Ial`hwxk>S3Hm;i4IMvSgxnEE^}wCL8{tC%G`dkrAMQqFTFp3o zI~Z;BQevyQvi7qv%xy&@6evPwZlY6t#xi@OU+3NEcGFz88Rk{}c=s~3@gEzcctRPU z9Ikv#WqaAcx?PwePO`wL)&AbR*!;DX^==`uc5nKLNZtU37o-+u3S$pEG z4b9%8)=rs!GJGIe@&}NeMjd(-&r&Np#3sDU^q_tF`s`&;{X+=I3zOG@Tc^B(J1iD9 z27f!t^|wzr+&Q|@yz%H)cKyxwhnRkW`&{S!8L>N>*nuK^N_a^PYbV-=>^A-_(xb!2 z!1vur3`UDb@fBF6(`xs>95{-eUjF{phmgJ6zjYSa?*2zv*nS<0u}*;!u=m(rFu4&1 z?>cl8B%KmK^4W_;=X(<~;K@ua$w{yYS$lWCA7iUSJ=^>8wLym3PI}R@-YHx93=M9! z?{05?IIez4pWLiqMf^jCeeX6>2obBY*a3FmjrzWz`*{Qty!~bmQu}ugRHhw(3e8zb z39o#E!2zLxuhGa~2c(t$1m5T$utJUYqd)UW59t#Wk)n`>kLdkC!o=m5hF=+eKmLie zeLo1sU_*@bUbt-hAW1P1^-Jyt#q>&MoHlbQlzfd5!6=l%tBiDG@pz!@gjico^PvCl zy#<>&d~ZLQw&<-nL*x<|j{KsUZ+5W2f<7cqZZ^Swa5%HUks0;EZD3&P4=|QZKBQo+ z2mf6k`0q*zT`uy$hxr2s=WjWL7hfV?(lpiA<{&--K!SmGDAF2@Mw;nyyEj>PSmOIr zgX1M71AN)>;(S%JqJ?Ki*Y?{k^)r2YEaeP7T~fgMnn$#ZFm+)Gq2&t*Eqz8`il>}iX+cLXc6nS9px{TzQ9o8eC?5cavn;5qh33ZpyT zg_`KUD|d>c@@^Hr>BuOm}A>Ml%gYjBLeL{;|da=98GrQWf2 zjOBgIOY(vj4~kPEjAT$#j>WP|Co(5695Q(B$esVP%tQkrGr7QHyNaXyYc*cFVRP*7 z?Yl+CalsY@2e4^e)v{8+UpD5|37%siM@2^`XdKOIPS`faM1uK53y0R#-{KITI0)%U zgp4D0r%69LDFABMdA9vL8870ovTWju%#VQKqIJ%^^GX%URRjx?QuWmI&vEA}tp*xJ z_j^i#3e@H_+>67DOJqEu34E;F-=~q`8vqp{09XG~m(G=@W|0!SZ?IQPGq?|Cj>mkLZ!O(+Pf002;D7fCqcSw*EMbvPuT z0eUyJ^?Q)!2lxOzN=<68gn}i*Dx7{Yc4!}~)z zBqF8E4V01y)J7Hurb8h0L*Vx)z`0Wf6{s>mhm!_h8X|P?f&#GCoLC{Pm-=BRK6Mqe zYj?*9@nTK)h9J7#DG7tDM|KM{@C)$=DY}JeGl-BlYN#T4tHmRsH}Y+T*bvGk6ySMn z>Y_~a6LEm9Ac?OowyZVr|0h`nKz4Xxy5dtqs!3Ez2c^zag06fL|8GbX1B!4vlR%d> z;#qaqHiWYd;dGJ6R;y$MPGengWZ;I7;*dr;ZF%b*BRyzCdN&hGW^p`o7GaK<)0~#V zY^e~gv|OH-zg}fkF|+25GJOC#zGwxhtS7=xVof+|nu}(w5D3DH;^TT63wJm?il9~{A9O1r-Cm@<}{&c zX_fZz4@rJ9Na2by?}P$0T%~v-A^M=kC0)7HAF@dhgpY;jC2t4r0jJnNvh<#31sE5E zsB5RTxaEd3~C72o-IW$`VBA2bzRk0GLj9pl2r_ zFPdgjcLdX2>f9n@1e!5fr4Un9nZlz<4uyAVoi^q?6ebtw5C*f$8_L_3bp(}Qx1Wad z2k5g$!pJb8~KtnE{urx`~ z(I+(GYx-y9xPGWdGqvr%5y=o z&`j0D0vUIV0FrT88CnTD7d00g*%uUO1Fm`RHM#voxy^;R&Pc}fE8@R`;u;fkZ-Mpy zNP5J-LcZ zB5`S&(I82yu7NNLm4}AVkd}&nQzW?pbF4K%Co+=K7NilGVVp*iKQAc=WH4P8Gd~){ zxM-1kFPbKj_9{T+^_s6494L?N|SQI&O$I z;3aYeF6AN>z`I#Gkadx^LF4jRpdP4m6+xhESK=HK`bkR)*dvkLa@yq)02CQix;@2H ziHOl^Y8*d_V3yG?K33`!##%-O=&bmtJLC{4Ac~~$d>PqWHQ@3oiXWE<~p-VREL ztms93r^|rgDGX+j1meLIT01^>6gh@)1b2v6rrlEck9^3vZpu7OLrtJXb)32~lGo)I zT9S(Lf2Qz23O8$uB)nAMUT_z&K7-dHbN&{Fu}W3|7$~80I$a#|d}yb1J`iJPM~%?1|XO84U<)&meJL>TIQQCM7>ke(YkjA0qXW$8Ve2zD`s))K>=F*z0- zA;6)UigNgg10*E}B;FCZfRTpNbQneh39<|b3>|m6T}s}qHanymD0XDoev+Sp#QPp# zJsv_t6G>Q2WQ}}+ojUl{8X=*WU`d2=)d&Qda)L35ln$Y~Qcf8lL1+b~#_O8M8FJ=T zNnyo*kuwb{E2reAlQ3czf-9v`)oymRoZ$1Q1|~!SM-4~OnYYATB^DJ2TxiqpT6G6% zMPgr}<6)#MiS;~@qEmP_IITi%0WcMpXEzImC2@R6CNT4HtI^)oLSgcR@*%8%HSt0RU8fO(sHa}Q3T2u@WqPZNS z6r{w?S{d*ixp-7_9hbJ{Y3Kd}qxzwTqBzLZ9@SomDlQ}Ei$?eoCPSJ}GO=ATBs(HV zcy%C9)|PZ&mtJvfp+BkUa!>H)$=T0`c8ahZ#5a(E7kAK3bYsPw?=gLAE zRdneabfoNm5GJOh8Bf(sjn*q2_X81FD@0X`8S>vvfU=pFc{ydjWYBagp^G?q^aFOy zhw?3m`2a*Smjz%45MYj=fs|zAfk4xOX$Q1>;6oMHnrfOxjLOR#wfCrFbe`Dd=bwHUtQ!P9t-6`RupV*g_IE)_ZFkrRSh9q>3>4;{?!6&d= zYdUFzqah(6X;m6tXz9;!7=?XS?S_P6FKGP(kxf{Yri9ljD#ycdkQN?@$v#6bC8G{N znUI5qL7t^GjbzNGHPKPywhQRXhDrI1gO7{G+!27ne?=x3c4(*+Y*dzDV<-wYgH}B` zb$zu)!T;d{0Kxy^0{~b5J@SZn;DvV+J9^WnTL5=c=qM%tJ8J@h43=w07JgxB)n#bj zr9~17#4Va7Y;$5(j70fUvK0mLAuP8euK0v{LdbC_(xagoj8ssHl`EIPXND5iHG_Un zTFO<1NHT?1BPraS)^kFY`C)p$Alc!oCyGJ=w1hfWm-aDpr09nk$VmVrk@L}U2z`!< zjGJ+naFYCjVIK=wzI2B`E+MZ53Z8m1T1#YBIh0;-a}hIW-;? z`FcyUoU2*(JJn^OS8_PAmPS_sTy!dIfPIc7R3YXeDWdxC6OQ_NSi z0zWj4mP6n+M_{9ZL!jEHY?zz?_vEGrm}XkWZBiqY&{TWZfe*8USw#Mn={c#iWG40L zCKxk_1esK)l#6pCDRrG(XX|&`WivB%b>$pZMQB;$NLMoP44}smIW~!-PLD9&lx96M z;SNMZpa4@Egs}LkVFNipounnYibe)>`0<)VfEv^vk0Z5SS$!mSb!`PKttzY{muL`I zYy?xSS7T6<=qX*vQf5QWj5Q%Xp~-T_S(xfyS@s!)BZY$i+h}tTkCoXsDx9BL`byQl zBvjC7SI-|Xs1vi3Gr?vy5$S-}z-foy4gfX+S0w_4E?P>jrG~|)l%J?*`!2y8aoCC% z*gjjg{czFmF!BLXrzcA(C_UGa1zFodC^o8<3m&*0kXPm+Sdp4>CSXCjG9uJf@wqN| z8mG9PE>(A}_X{C->=s2R50~~Il+`d8jd)o>byw(Pbw3oM*)(&sO2tc>r7d_QD~`Eg zb3v0$aFiC|Q!(&^VFr|*1qL^Emu{5Ep%*)8`H@=o0f0tw0f4HLhee#it%1ldLWg0C zM7>*>ika^^u+O&AexlO_JV9}5_^|KYD z7a5cA=m!^=a7tehBq`9 zhEPmMD(zXBf>Ea84`3i%0p4ek@JtXKNimI5K?{f!$#tlvol`7i^!##?01mpAo3VF; zaXFxJgI{T0l#`oc)6ZWylnP^MC>K^Xp(Qh>+pDE=ap(0V2iK3`^-e>cDpi|7gzgUz zu?TVaB@_LYrDtk_I3qE6reGax0}NjHpf_ZAFsRyF!J7^_-iKBobqb)7QioZB?KWE4 zN5zB=qTD!^ZHw7#ad-=Buq--Nu4mfAA(yTr0L(`Qp@$+*5hPS{WHltT9GwE%h~S)} zqe`B^33j3oM$`eG_66aZRI@@Vn_cJYHO3ihRcSh|j82i;R zW0JfIR*6xEQ3t)RMY7LzoXJqMO3YW@u}MoLr1!a$m|hQ*5R;j0QS4H|%XTqxrqQ|D zYZi)oXmiSLJ+FqHwq+%VRpx6?gfj{{+Ql8wl&vXUN<~i=R_}ADRTeO{ab}+9m6dfq z1*2_t*=V9Ah@pnF42wlrh8FjwGla2QVvCC1W1+^v87IP0Y}B+_PI^$2mCJ!jYN<0* zDx*UT#uoHNN?PDtVkf#}s$NohVFelKvttZaPjS@b-W0TL2d%M5RO-(Lv802fu6&N? znAef30-VZpB3dCxmF8JZlRc?6S(JiG#~v9V#=hJ(`(aK*s;5*2B8eZWY@UuvR14b*j-emsRqQbQ%29EAhqcRZqb^n22a-zqxe1jS3a1ujac3Ostxq+sk9>^$TBuAPHzsu} zifW>)o*ddTRBdB+z4MH2=rHM17WygmQ!8o8j;WaonAAzuv%8OrvU8TB&uhSj#q3D^qi-IK!q}QRsCIHWtXT z3|A&>gWHB-X{m7s(XF{+)g2hRu5?iKU94DDGBi;LnI%fx=f&5H83(SBJrw9(lCyCN znrWk@E6p>C?y8ptF`Z)Y-L)dYJn%EnFy-od%SzcD2u%&x1v; zMjIu>%u8t2%GbSJHPvpa?t~SQt4`C&^f97Zo=d(9S7NX<7iZM%nN^1@xSCN&(sWy( zvZ45aT8lQw!j+6p({wJVK6AY$XeGym;$%vQIvaBy5vtou&kthj2~o2E6-hgDD5Ki; zx|d~A_{8dZ%M29r#J%J+p7+8F0bnORd3Jyhp1F?`zfA(dT> z+%6ScE_b1htL9rpmu3$4VYsFZG{~%^9@JV!Qwo|L_q*I>nh3?9=GJ$4*AyuWj0$y> z(lw2C-Y9FOiMncqGruChW+cq** zAa)?h4h)LB1w{&}K9s`Fr_i>EZ2Q*@jMGRgudr)tszq|@Z~zZU9Agx;wQQ^5OJPfq zq=KpLat@1OMOuuS;`F{3l$4IPT17yPAX}t#X0d8;%-EuYT2%{EhAE`LDvV#*o+rPuBudI zxE!TkAthkbDQzPF018REToQvP5^Q9Zr7VjrN?{XqTLUccLU3wP%8Sn1UXN6pcS=XY zRf#TaiJdc{21HdEB^Of)RV^2+)qA@aG|f_~c12YaVRD zGZy{wcO zBgyMKF?F(zq1^Z;6*QrXNi&V>he}vsq+;F4!cNsTS4G&B=%w8ekg`@NwXCx7eE+V|nfXfqn<60SJ$j z60fXbODwKjdvdcAfSCF-#Q=8va(Q4tQlDr`q!xq9g`j}+&*cJ!7lh@o7J&RRi{S;s z=?5r}2q45S1z^Oqb`am?yBjA2u;?Kg`g6N?-s>DH_xa2V3iuFyJD{b8$W*AF9|F!o z=p7Is0u0u``K^Vf|Gt3J;$Z=SUBQ4T!REcU1c+}G2LhwF780ux9sxkTA{(iQwv?VMS;0D6Nt0S2OL&_m~E7#^JqS`awkxdmqL zXdIZsZ3a1R_>SMjljsZ?v2OtTe0CFXvVc9a6s&mWjyH=Kg0n*MF95-Fhp+Gem&tL3 z!|c8j^#gtG{A>V+2dpEan1Q3}?RZo*xcb6UL59P-^+*Z`JX=-kD?`8wQb0SLQe%X{jXOSOguKBmsjqJxo;}qI zAwhii316S`MnSUerj2e8iDA~Z?F{b|h>F+NVDT7?6C_@L8nO13wHlL~ihVFypOR&#{c8o8o6DDxR&_$a`}O!7_;Xi4Ho>aY7p_r0 zP6$Q)j-oUG`Rx?=jGr^mzI#vrTy7>l5heVQ`U44s7&99bE-ynw1bp_WYg!O~LGiKJ z9DqvTg&=)5SPTY1zyldz;y$QuPLDXfrm|C~7&v&H(Apj*jfxw1{hsq}h=>54 zH-XbfywwJRaa+hDt_kB(kj`)1aYk)0d3W{()2F4YT4<1CfvxS;fHz}|W*-43Xh(}m zwQ#Z$@P2k6U_=MkJq0sVfEAFNP%E~8nH5S!SgrV0D;5XM32lmtKH%q6~jXi|3>CX(Yggp4?3-^P<3^kB#Y0cW(x;@pF-j z9>M+olKOxE*p8QIIPgfvJ#?#?v^JjG1bayD&t=Y&awQKY+8N={< z)fFQG`yvveqjFpHuUMWfpyCF;U>54@Q-E)hV2+m>;$<`eU%+=^&3+p><_Gwl(PAK^ zk~TXqI9Op|{$0V5{$~EOyu<`YI~lmJ*SgRdJpamh66X*^P*&2*pludVqdwps>DEu~ z(uLBtg(;}s_q{Tf$4%M@BtJoiaNwDKLzXWfQuHrGx}}mJhNvw%yZt)h8)Q<< z7Uja84rCOkq@nm2AY_*^T>a}t{|7z{Kg<_sAam^>#`6hA-J8ykJzn|iqpz2e!Vz+o z!T>*?gmLwCo1SkhHy`p7UxM)KE)BhHkej`p4UU)HJ;-~A-eyahx0ZPI^FbkB#GpfgE&$rtxYQ`dQV*HO!%K~B$rPs^5d52A|< z8UuyU838mobf9^yeQ16b$1Zk_pZUP+)u+97pw z`&=q8h`r%e`Ey$4hS<{Zl!9TAMKlf+Z-)MibDuH}gEc-au)(CWqSb=oq>x1mUH99^ z0KWGRjP6NROhN(dk-R?-7+>||wg8ys`5RZ)7tDS(lBgm+yNn1f6{>t11K)tVQ>Qb%`|CltL>?`1E?U0Y8NdzPUCRK=ou#P=;8uux z<8plcY!|pU5pR+xaKYvpvi{lY1-~cs^-B0ti_@~fqxw!4uU3^GFCo1H0gtZU2qG5= zCI%aBH&xyqb~8ly>5!s5$)i@ZL;{ZbY~L%(Jgz!W30TaP9YMl2tfKJgs4he(B^Q9+ zK7zsui{oWhyR49}Jw&F67h^*(rk++d{2pM2U`_gUMR2^V{E92cRtq6vdHe@7^cYJ` z*tR~52|~zuYULF8#I__V?asx(y(GBfyxM7+z7a+aM+Bc9p9FRYu^nOk9X`e`hRw5o zV0^d=XIQx)Y(1tBisfOcKh7b-@s3!~HNQ-C5BnBE>pUXdSe@Qq9ID}cQV=;UUxrA^ zHNCTqn4tv8`r1{1GwJa1;FSh0LlnqLTB75h1v-FJAKG-&EkyNe+RC+|j+h8jA*5%- zz)v{uo!XAn75_I(t%RhiM@;130T@(4EK#mMj@Ub*SMSYVri3CQ=B}GDyI&B`sQ({G z9#@38B$1pI@r-yM(dCQyM35QETM`7Gkbu86U@v}RM~-uR`5#4RD__u|-_KSQ-1A&y5o>aRayKLJLXl^J{r21N4Q-;*-_3U#N*4pV~wCszF!SjR|X)kkxtzFvp*&+;fl#;F3hN4VYjtc*Z%)`YI~>xKlBLI={{q$6$% za)Bw3dy&Zei1Q2bC^hBFN#h@yJ(&J9GA~D7=g4=+m{yZjXhzSiem|k&{mR}O%fO5x zRi>ADHENv-lwhzhfD$4;*KedwqN%holMSD5WpsbEAlJ93r@JoTFN{Sq7fg9I)nvGD$*gXpt># zCp)7|DU>0snGrB{Hzt(-#txMO9<9XYrfu6-T~|~ID|MJ10<)*O;T<(|j|6zO&io=E z5`K$n!chyJ0R#a8p(vytAdLNHNKxBq%vQ6FC;AI@N1p*yk|u#nbnfp_ne&ADg2;|b zQXM1NoPIz`^y25EDIGL?k+Vsrk)DYrMcIB*zb4bP4coY+IbxrLUFP_Rq|H<`JnJ9wP+ zH$qD<1!tyBy{c$sK-Q0Bp1Wf*w52g%YwQ@jU-iz-4*K9vw3QI~-PQ>6DdDW6Xq&Z1 ztUF(MYA;8y>K~t=?-3TjxvPTH7=6K4oCJOAvibTOg~wALqck%We)6u-inwNO!>Nv!sl*bZr+Lvs7t>F7 zmEnS97;WgdShu>9`}(6+5?`<94^h;8TQDZ)g-5J@Xc&)9aUScWQ6IXH=*qFGpMeR; z@AM&y-~J+w3Dl>vL~x-nTV`u=I#m1UHB{i>DlZi<6;psUSgha-%Gh!%!9&Ej`^VDvB$|-#1dCBEW|gxI0e$NNbF54m`=8EY%E@p#Ke7K z%>-(S#hj9YtOFLSb?$*gm<^S{*DhE|Gr~2j5k6q;q(eICb!7PMj5YP*+XOqf!a5r| zRPbv_btRiHHd=IG3o?wYOxhi`8F~U>i95-jfryi)>0WZM3(Kf19DJt1rg`XsC1GC> zHA;@)v#iNrNBFEZTTO84U?-Q`7x6E@9@yDOCV&lR{aaz9o8jBW*d->xU#!&Fy;x@z zyN~(98)V%%ZYyQPLA=u=M>L-oDz`5BO}9uEE)MRz6R#hPvlIvO07uOE+DSUgKm9kd z=SF$rFlxm^&6vQsD5}tk#EwGC9jU`GfWmYzeib)JB|mYx}@4BC2;f zg?z?Q9pV>`<7=eT8Cf{q$l7=0`NQ$iae5NB?}!r-TR)dZrx@Si2aW7w{Vqm1&L+NR zAjJ1p|bov8qT%8=^opY6=WP&!8XQUGr=y5}O5>9d^S(`ZVQI1<8 zLl>{RE~SkvH;e9)>D?W71f6bkBc+CuyOZNC0ElkihLF2ZUkUE5rWR>K)xb| zQujskE!tD9Tk7OWz_UNZ=9`pKyW5A9KtJA)etV+>XZi`fLyBbcEc}ZTU@v zL=-ZIkWo${!aE^H(91li8zJBq4&zlm^!Xb&-jH9#{gbVPI;lAX_$#(s{_IZ^CA97= zPsS;7HD!+At4*&eVf5$4wNNB)$c83Nr1Cac>3=%H!5I-U~4 zJADvk_GnC05n9ryQj4h9*cc0wZ0prti1s~xmv3)f@R*4A|2KQzPM=ZDiSw4+*t_?q z)z)&}@^&Rxkjz;nWV)y1D$Z0wkx()BAp(-b&IY0MsL|G1`6>A zyV~|ng~XE>V@Zl1StkRI4b0y{-IYsEXNCCp*sFkv&u@oe$Hp3pGocp7CSa9O~ zk5sn!?fzTjfr5@BBGnBA&)^3;XXzvHYu(`>)mW9HlDZR4DzFMeBR(KaBqJxagNZaV zfsPDH)y^S53qN~a$V(!EMkWm0Ycc7dK*_t1bpD>HqA_^=(DhAbQCcD&L58b< z(9`U`YO*w}U!-P!h}_33*~LfumN>!L7~I^j&Ui}5e!ziz`F7mVD~lSA(arW>6kSi` zU;s$jc*H(dp}yr~LdkJmZV-^8TtAmjgB|dEw*B~{z)>)l``n&9_A5*ZdM+`!2u)03 zpmiO&ekpwY=%-ONGyp4L6qRx4wJYzTe>Ly&gVJxmAjK*D$4X0XAc}@b$fMvLrlWMi z1GpcL@P*i6)LT>aBp?P@%-Qi@Hqi17AGOklRok*mf z-YgYN;6YXt@fRd7bmJ#DzL=-P4s^~^lkRHlsHVW52seRz0xW7OmB*$;GjmX#z=nj$ z@$+=@;9FTwprGzQ3Zo!@ZL2#O8LWl=!8=tPuzGweMU#MQKkKysE9@Bgez4j z7+yX_viTotj+KwJBIWs^*O~*#$HFEuESwx$L>7bVBF#aaTShhUrm3XBlMdt=mlGSv z<7G_G!C;2{;=_+Q=KWtZ3 zgnLxCZ?l*Ts;BqRJvA@-MsCyT(10MKhVrhd7-8X-hfC3_QzJ-5Y8?31ed{|l!=y(W z4sX%oHDr}NcVBtl#&zmX)Ygl`S}#oJD^5x-;!^JE2%#Wd= zXO~S4*?W499~VL|BxF=psSF+TN-mIxvIB*i=J@_lm}SB2POlzNAp3E>q~jIg`*@1! z_4)q|xCwTtzH0|pWExRSJ>x^~6SKubAh|t1U0sQN(Dt)LP3iC+tQ+-5c~cQj0-{pycQ5D_#qd~vCkXu9S6^G>BdCP^h@^uu>0jQ0eaDWEY!$Boyg zrGm0}<5C-Fa^mQz#I4LkxGG^w?0HOPT3l0itLo(Tf^nDJ+KKCTht}*?b99~El{Uy( z7}h?tiOk5_^9yb2^6PKNV_Ef8R@&;lM67o~JM~!%<-AB@c-U#NdcdGO9-fS0`iQNd zA5!{M(?VCi&T+HeNYJMR7mp6ueO}WorR8fq=OY~)#_4y+Mh=xuz^$V2v4nI;gDM&o zgpw(O%zZC(BxhZGNBTbV5xPoh3^|+yU+{&#eSR@MEv8FLG^*jtWN}@_(G~nO0gD|1 zP}1mg$U2-1!NK0&GP>PqPbL)5gUs%& zfS>lmcILk6Ky&b@(tXS~A*rBa9dtteN9IQqRehO#iSD!+bEkkSo%P-CdvZ%|pZ z`0WIuW7wsy=UDe7ukUA(fccx)4A&_}hIoGj^B8^&(xV}_O61dBZA9)$KFV`$aiN!*$@|zr(y+%q-ag3xFw|wT5?P1?#$_$5(Atd`HW~cL z%+DXytu3am+?M^SSslGI{^o!!(A;79ustS*>X*8QbQ2E0OktEq?sjg&H_TYsHL*7z z6I1C0-^~&Yxmm8Ejb309Oe5MY2jR&_yH7S27+_rd9m9lH{6GL7dMD|Q z1rz&}qNnYOvwY(o3)$E5WZGYB`2W+*6XHgz2nKj_L}@(GEKy^eQG+=f3k%D|d+d8H7djt6BZv zwm-o9cP{kFiH2Jxf46>4p`fLwc0p zDH}5vz7g`@g&7_nHYs!Z;1?OjCjWaJ`_r&~NUqZxXBLnqbYt(94hHxji64IwIRF)y z$-1gA5QJYTbFyWgIRYZER75~P0U6)^=5CY^rzG87^_OU^n)k|PVi?7SR5k~9*s?|U z!aG~}n>mk_ttR$%^!OreL2Pw4&(wg(A!BQo{GoZlzk+QL3bx_k3DI(A?BH!i-@eKY zRrKt*511U{2WZKFF4R2GwF^5M?w{CJp9Kwz@p1PsJ2v{M+8Tt#?z58v)emsI1qz0c zoyjR+-cojMo%%E;pv>*7B1!y7Y0ZJQ$J`QNO2M52Z~LI95C1}4u{MNWpOz*ZByWX- z&Z~k@bR1cjs%pi-ae$IIFcv5MdjJsMe^U)PL=;$s&is(vS`Im2H}FFqS(3hmg0G0f zU681F$A6;({rJZb9ito7K}F!q(GRli*K!OVNz*x8Zh)t8$1SpP%-jj(A98GvUwwXa zNyV)}aICu31@>6f5P{W097o?MgqI7%EmL`VeR5pynm7;%E%`y*18lY@Ee}XuH75{} zI_QC-+}0N-Dk^BiiJ!SIvxgN~J5Gwh!QtE)3?y9x5aC)S-#JhOhM90O<_{A4tiYTC zvd;1hHhD$AxGc8D=K;WQ4d2Xs3dkwT1Nmyqer#Jf9_SX_UaIZnOyhJ&YkX+&6^el3 zVqCsje;Zz$;qeeah@5g}o*=S%PADBV-zEQX?j;V)vEd_m z*@;)OLCpD}Uk-ibd@;$w1=%QyxX`ch1U+_WA?70?Ttu%Y6LB>t{grti@#YF)fMR$< zePYX!@GCDUpD$c-t(GhDL`$w*K$=9v>IJ%#_mDSC_dFkAZUdJ8%&EiU?k$Y6gVfj8q*Z#VRd#s;Y?pZ7YgJ1a$6AUp*A+yxir@Zp@is8`_g1%ghT=UD zz|DDf0b-oqM*QCQl<@nUP{5wB$nU4aA4=#8JYA^BL>kjZXa+9< zj_CAM_?tf&6lVoq9e^bgVBf%R;!<{C4o9joIXjd87Ui8+7%Qo0&Q~RP6?%O5+x!E2 zdWyOUi$OQJ#jCVcXJ_x;knpPDfRO!%$(K@vS2G^;)lC3d2wv!v*T7mVB?<9> zvo8u|i*(3Wn)!xs23MbV=J~yrGasIDz_4lY4|%cYG9Q4+#l58o@h`t9hisrC2*TO_ z0Frq8D|FCb;~j&B%NYm=XJs5;&nIm8_jejVL2Nk^FM*5)!JP4l>agjg=Fa@opXVGj zSFEWG;PZo#$6XG5HArjAJOGTJ4u^@=qX7Jee=&;QS-#2Ze!N;2Hvht9NX6IpM=2Id z=P|^ehY3m*Hg@ zKNg5Uz~}oSKyiVAbV_l()EL`mBFHH~W1$9^2Fz&$3zNc@Oaja^Gg1M`q#u8rhj79- zD!6+H^f(FodVL`Ts@YlZiu6G_1n(Gz2Mq~DUF?+*bOJLeE%%Y7oDdfYtc_^AU$*|s z&D$!7pqR&SG-|%@F904P>|8_zZMieC zA)##kTdqhvn(%!J`Z2;gqMzZ?zOAF!9@ON50R|2S2o^#=yGiM0@qo~o&3;^W4!a5N z>|7@cgY)_%fe=8=53A!@JB#LRGDd}f*&X3o_*H!Ig-A>EsH7^3vd@W5^t&`PyyI7d zg{a~%pzdQcw9x;#gsV{}6b7--(61GC5FlD#523XeEPL(An9wP5yo21NB(6L56`07A z*L_`!xi1VZW}OYidZXtV{ib1jPr@`AWtvF@2g3A-v^SE`;kz@qL*q|av`Stn0tCUh zDWsVrd_kKf9F7ROR1+pk`w(711;F@mf5>C@*IPi$(PW&z|MthvuSOB{pC7vR%mB3r zF%(oT^^^TzKFS*+RH;2+a9n=5_l;8Rd9)&Y9OmOk&axkg2(e>8GM zEROR@5&9#M&&nncSJ1ZQDo3bYUB_e}r^tXs#q|RLBv<%--Q;nzdBZ0z_qnf`FKz^X zVIrPjVJm6Ht;v0oq?SMWW*N7mQi|J;QgpOsY0@|TFsVnxS44L|Zr^5w92pV{lrQTf zB?T(#ZsF9HRsbrI=euohBq_x4rJ}%Z#Myx@m~kjAp~xu%PVPb?`>jmLMLzu=36mHi zo(0)S_$RsaxpgU+x|~e2sBqlE-5Se4RNP0fAhkH!r+ABM5X}5LncU2^>ap^YfcI(= z32W&q6tCm`13~cT+(C%AsHeU{lvs`agrj!Ms3TM~w9L&ky(7_R*ygXxvpfUQ+=GQt z!({rfK)b0_MoaPDk`wp8#)=)Wf6$*Gpps5NI0S|`;#+_Rb}Sr<52?ZQx?lhLCjrpe zkr+qLj_$1F@q)P#QbU0YdBpV z(M8M)6$*}GK#Djgi}A*)u)qCRZtT0ZVwnwTUmR5W8|J|8HLJW?tAA1x}~XvY891;NUZ} zjVUN^EDiip2JRURo(d*ioNR68W|SRDso;wj!I0_fGf~7!o9?IZ6yoh?)~?}$X_N`^&o9^y0j5pZj=;5uD7-Q4Q56{kVJzkP~tiZh@3 zo(v*X{1fpnV5^|96^BCBRtc!pfc8UNFZ^2r^QQ#&`74hLpdd0j&QxP?d?k8$72a=` zQb~Cg0H0+_`b4^ynG4?#BYzfD(pP!@=?K_5m0gjS$n;zzYy6jvD87IHpmLuK(lTVI zD=>@_yp@uGuLG50QOc%y0Os~eg=Ak<$_x8fNPY45*_E0Eqzg~CWJ>d_VvTNJ_N$fY zB6HqC3-ncb%_CO=s0}Ujd4uB0<&1@Yw7*{gu0hG?#my~(rDrNN z;8hoFsO82Nm8dk%P)*Not+EI(wC~FE!>KA3R@@h%s&WY5Yvr=(iqg7GDLflh>ubAG zRpYI{VnLBz!oZZyU8?q<0}b*@a@GH%`L1&H53~}o1kpp&S7YhZK#eV)Y;>C82A2Jr zBcxvK0PYd{3{Tz?ogp=qQe6S2-mq0XE0xb<_5%*6)n{flAGPwXXD|oC%s#BzA8Ot6 z28?{A?-&#HBfxlX4C(T_#p~AxsUMj)djfS_L2%YW{UY{ruUoxl^1WF7!QyVzjH$oq zd8M=_ip%nvYz=EYsLGkSq#U$nrvedy$VKXwc&D*~#^Idqv)9#mdo&!8O`SD!ug zCK@L13~~!=z+R0|0|DIB;a7K`Z`Ek6#WmPXo+5|0BsIGd-o`(6bnT+ z{|s-#KuCW;@rLawsaS=%_#iahs!WqnQw`$x{3{$*Mz`OtxpmA>`f?XUC#ZQIx(z!0 zepAhNV42JXj;E-zd+nlpgvPZjk=NL*#vFBC;>OLBqsS2kY0pQ=?Avoo` zw&c^(S#tpJ34Z_;e|2_yT1eZ{v3DbSVIULQ*2Gt=EYf`*B_Qdim%d|Z`?rv#p&{*I z+VBzR0MMWaYEmp~au*27xO>GHc&(3Md9oN6du@?Yu@v&eN>jd`x z^Hz4gC6$HHrWKz4bUr#!Yd8Wa@pokl{y^+P+&ZC7IToXvdqAfnN4oZeFI{-VU_qiW z13qQ<<@`)!WYU$;{sR|o^(M>AOVpXB(A8^gVOw`6;D3AQ=$9L^S=+6CE|Eb`_0Ko; zevNBbv7Q>ZfHir`v_hz-XZEE{j*S0>bUU?h_~?1j7T3Q@#AqQT$vaig&$!p`b1~?% z(?lNmdxic@e{k*or#HTL9hX0%cZ}Kkr+hsJ00G}Tst)@&&d}D!-23DQC70?cO1IKn zU!QUr<*FZP^q;>F|0(r5;`+MVN_^-WFi{8g^ssW-A+R+d04C^vh8#m{8S}zf4LCL2 z&BFxVBnA6O7*hujFp&2+DDYi=A9m9~UlO9wz}5Q=JnLJwS(d3HV#&!Ef(Gtto^w3e zJ&eOwc*gRUf(^Vl6aUh>dk+Sw$#?#<3QwITaWBxW85H_p2+K3Q+$xyrRV?Q>G@Q+u4#T^ncOa6LSVkbScmMtqEc#A5 za9_{V2yNMf4hcD^>5@L&%bqsg*SNGmI895ypAqj@CeiaQ70*aiDi{ALtX;;_t1yjH z1KcyNA;JtZQAg{8Qwh4SkwO8D^yPQHj*A2 z7EDSJav(-Vhklw=D=jy7Xg()c;*fu#ApxFx!wZAw^QlWV`u(){ZG1rOoDZC4J6CI} z{fy67bh~F72OLUc1T^<9#Tf2y(%lM7$!3g&XdSW*c*0M7$P|ivY)oc<-b4BPk>TQH z<%qdA1_^ebkum!My=F)bBQXj{jHO#-A&r&NpHx&$P&*>%voD6R_SNWfa2SWKkMxE5 zdJ%v8mXOKpU$*U{p%(y>8v=>tSw5% zM&_SE!V?^r#>o3Sv>lIY3V?$EAj7ix){A_<3j@j7WQa5Dh8iS!n%;Q z3(rvvJ5EjSd43FdE2G)@-%4hXA{L)1)7D-6Go#&$7KT-5Cg&4o7v>J;ortk@IWx^i z3Q~yJ<#Ho80W`BZ=S-U_%N)#_WY<%dOEib7l_G5gO^Khe`33#<_sCUG~H7-lBq<@@TOKXY@{lg3?0O^)cpuFN=I`;xfgP4U6aR#8;tVY|)ug629&;vTuSyi&;)+^wOO; zMrHMMW!}05&+F$8Sv=$uB}B7C^2R^K`?TR#FJz6cynR`sa3t!$8|{H5kx)E+j#Tvg zOJCz{usCyvvswaPcrIkH9-c&QZ z?VFXI?;79C30lSXnvC?eRg@cFtGv#?PyHTbc6@l2)gr6Y5KmfUN(8=)M~{JRbu28Z zI}ieKiZrW>FVq^D#}ad(e+Vu)OtIBRI_+=a`+lYfu9xG@4`$6mWRY>mY{Cl!BRPyV z)h%lM_(kum#jjIq!|dprz}omz-(wrx9A>n{2EW^Tq6&gN-0B%gOR!N!)6m`iIwAz= z7qBAFrY0|J#=v>n=A%ka!N>|f36%O8;esxOKnYEu&B3;UZ6JHXZ&1<0&PXugxE?bMR{neS06{RAAmWwwn{riq)gRJT{%ee zvo(>R>$!>8w6%8yW*SOR{$g93$%8QcVE5z>BCf@wM2i%^?~jd3D3l4`l9}~J&?{?g zdu)0Cca&{?UmqTH{WB&59{$meVT6bEOA|RSQXsEsy_D6N_@&4GOcL()Cf)Wct{Q`e6 zv8UygaJw2feA+)k3@l~xp^y@MK5?{Z@rt0lzfsd3!ymC%kwVZ;*xnGogDHh1a8+&} zj|SMMPIIy=z#`7fvWe9Jwg3O8>D4kKIe!j7#9)1~X4vXU2Ai(AVSCx&s=(Vb*p~8T7PycJBaFbWWH#Ja7Q& zuA7fp=!ju`HWtA@{CdHbF0-SU-HnzWa2p7mK}bGzl+SB&)KzEsQW9*Ap`&?>S)vk> zFB@yP8claGv&xmH;P z<%d1ebB$Bsc#=i6^luAi7{Oop`)b^OMXuvh)6WKw@{GhkC(r=xH*zU__klim=2s_P z@#-YW1=Q~ko(@Ih(zAL9L*^te_v1uGvWiN^vTaemz{&Z+3Bq1o@LKk8(dTb`)XsyU z4l5}4YjpLU@&*3T&bh3Ba$j$pyo6IP;O;stF#a-1h7qG1U^nCQaHImo*cSX^G|oV( zz!|=Xqb}?e&RduCw_0NI>U^TL*Iv~5{;U#(^X2d5L`NyIUv#d(8Pz{T8FEHlra@;K z=;HjS6z$K#K7x+I*$lvIk8@`?pv9vHP+@4iZgPdS3mgvf(0*@;a3QgE2o#GFy_Yh}oW4LnIvgUA%Sa4BTt9-q(&elB$%ykrd- zZ>Y#4RUeWucrxsAoWgr_jnk(qZWGL?!d)?f!{;nb=&0fMDfg)uuCf5d)saZzGw8B} zouLZ;*fL}H{dMJNNPkEdlE;42-d%&8ELB5{Kf)`2u5mlnO&*5@kBh4$Pt6yu8AnBJ zq9^6S&9yFr1}a{|MsU}2=^g2hs3~^)ffl&4>2J8 z|8jOJ?w|g!9ks#4jWLLF%T3}DoyMP=d?lugYSkJQe9WIB)dws7ldmsaF^=xE1QcV)N>1+zuP z;oZB@YP)cBN^jly+sla*4KQ`MechYpi@f}#JAt{TDdMYm51J-POPT)y!}C5~4F-aE z_8#}=AFfs1E2>DpKe$(MCT?eHgR}Iii7i)jl1}B`D}vqMPrSc|m>hO^BRAu_x=g8P zQ3|~&x7~R#4uHLBDsmoI1p(~SBYGuC=goGf86RKyzKxQv7V{RWd^PIrWO|Rw3MbWj z1XGg;8H{$u)*7^)=>v>*hQG>3%x=)baW~tm>Ct$6kz`x@ zDD#0QA`H09(7l}<6ZXj;9;je<#0IXQ^859f`Fi8z*3%t8gw4fm!z6`G(>M~iGm&9}g%H#8qYw(l7fzC|7{PmojvNx87NxO6A=tpLrj?*(`P z-n@#U$5NRJM%sYO2z%NFd?qfGbNJ->? zci1l(3I#_aiVi@8E)M_M6?y)pPPdyp%G&LIzx>g=|1R&SzP@h#1-ET{BWdNo*F9)# z+WUj_qJ*s)8^B|Yw~=B0tCB8Tg|+wM%oU3L@quMG^(5;3aXF}UpkSvLzb8%Nvn%7K zKNIZ4l*XAq7i}9LkbY6H)<~1O0hm9RW8iP5^R!;ihQRM#Otd1hdGo#SkLN{y_$TO8 z5diw(0k|YJ(cA2LA+JDw*PVEQoJ=c_Y}hH}2mJtLD+*A)*&Ws54Dtoy0R2clh~1+D z{@UJZy|lkAz{ro5fPe}B)^`E&1Cp5hcwG7@(d&RbsqqGJ1&}me4;H3eHe=H{iTfF4 z0eeaR-6IJkHt|Bk(gU8V#vPg3+Wn!+8{2*ZIY0rwhm;wF8I34&UU+LrLDj3F&8t7|?Z(X=Xiu8v-X3B>V*!~Z#nQtdcWs|%c4{0d% z=N8!MY{Gp>h^Yn6mEA8i@NQ9I9ZSDB8;Qce?`LG}-_m*eYVgO&d%iOjQcR92+qG~6 z9S#2GFhtvms^C&A64(wz?;{B!&`%K|KZkg~ioR;s2qN>f0Xf8nG8usksMw~En!_u& z{vH9jM1vos3@sgh$XFw@FbvZ6kNyK%IRF)yS$C>16ovm%=HyinMU;7dWVA95g7EKe z?pj?HQ)y4gIeUM*4+fLojGFUxeqK@j5M~C7u)yzQLzw;D%}mTEt~t3Pg&-Zy_xEHO zX5m%74Zqe6f8sKc9oH`07HXf_n(e z;r_=9Lj-f(P~XHKMzA?`aOLYL%o&3ol7}$<%oXcbFq*4cGtOQX{QGfXq03+nVPAz$ z&@koLpoM`q9(2Dkoff8T5Mkl|2^+IZPw|BsqSkB3f9q_~A&X2}bB`&H#Z3MX2M}Y6 z1>w|UBsAs2Ew-{W;{z7k4O>F+E#^-zXD@TI#0Rg(jS`g{^bc67?x2Fd(^dHr$j}XS zmipRa#LD8SST{rX3|V^m2r6LXAMQ@z-J?Mhf3J{Q&o9HkvXqug&E(()rpA4TmVFGN zstWb@CwJi>)JB0&5C{W*Ye^xf*4sD zral{$Oa8Dqe0H6%vOt9^6}k4bss%-Z@x$$t`Cl`=S~V2K&1$1R!XW{W0;>l_o1`VO zdJns8*{(GAYZ&P7d@{C?!8Pz98!)_kpTn0myusd&qQiR^N@Es!-C*|+& zpY=s^3# zY}@mkTe5dkd0(uq)zRR0w&R7Q1Kxvg&#Uc%xJJG0zu92w-~uE5vO@&V{$a;PeVrXqdfEQEDf3w74IR*D_OVrjf>N>})sa&(7~X{MiL_&<%!Ha0LPQ z%RIZNd2CnDmjiMJ4@}~lc3?y#|Dy;y&gT9(O_qh1MJW1esit*Vtp>U&#bqQ`$6&VJ0u+FFWAR& zSlLd2gUUJue%knK)@J+9K`RqTZT1E__;!2APB(Ei1k>4avw+Ly;P2m)umi$6(p2H# zU!yTSZSf@=3+MuPen&@EI@!HLlPV+k-*zfuaTsf<7Q9Y}z$YE)`w}>sqI{g&=YChS zZdILogTYmeuv&o!76vDm;N1w@c|5i+tma(ZvBI6lb122Pfs{DL7!&6Spu@}S8`a`* zahj1abOMeX3-Js?vNPulr>L$zp<{8VbgVHQJ5wjoamPI^)Gu4&d4!ja)>dol_@rLx zn?JsdAb8#rCqIw- zf4KLTGAUlp_882+Lby3mFVH!j-~S3LpP`RLe0PpQ9gnnfsp*{etagFeb_9ACC_OY; zVX2D_$n1h^mtAYL-LE+qzAtopqp27V7Y2L->-j^Lvsd=!!rxnW5e#O=GgxnN;9|f4 z@SuzNOO2y05J7A9sK$T10i}!kzG3~HXm_EzWY+~yvF>m2!}97wQ0@ztoR*;sxl6&{ za`mO|k(qS_mytux;|E-}1Hm8m4Y^$I3pPbEx|}%)Ln!DL>nQGOO(zQYC|rKaozE4k z3{RzhVOlkE5$SAqnYs&T7uFkAvjKsK*PJ1)T2BkiL?dr)j4YLRB-3Mbm z!6<3}BED?8`*g130h6N0AB>Og>q1`@tsITw@NM$8XYT72au8e>+I;^6UA*ORUA0Ye z%KXrP&F2&L;IReRb0Z+e_9f(qP+243}}_Z$;J5=-af6*yj==Mvm#<;*t3^X(wF({uB7dw`eU&alfc#}}A+KC+~Y z5b!`&?FEXNH|(kjl@U?XU94UNV16qvpC0vp1Nf}i-yAQ#8j~j=esHi{h-k3?6<$r~ zbJze`C4_-TAzQTkdGD^?yYpEN3}v179tfABlsZr0{hfN09T>8&lIy|&e7}0l^YWkp z2z#$d%PLd6mXqW2{3u@rglTH}fcxp zH!+5ddP|Cw*$-1j-#eawOL#>8KHhb$PphnV90mrF)MuL^RX# z5#2tS#6aRBRRqaOM6kah9f4YJ(8uEO7wtio!H+}!!dk<)xk8^bxA4}AN31~aLayRg zUH0dWeexZ|ZxdzjyjoRyvD@8LrGU}})5t_Gc;#34-p_+50-Q6(mHZW?TT zuKywW^FEZu1d5mW)oC6X4@S8hg zfxpi`+@D{6Ab;+q5zk9=&E1T{#h(mw52LWwml|F;!Yc$?srqv!y)?xX_zOT@6OHVx z<3z8M5LEN%Lt_ zLQxi=_a%q$u@|J!1N?MY6yU>p%)|%?U8jJg1C%Z<4P-VWuRo|(|T zKu|Sc9{Q8yXEaG%Gr)}+q^%>D|8)F8y6!}aSYFbLJ=yXdUvUX2{`IH5d;^|lVFAH6 z3;01IC`J76#Ul{lF`26zpdXL+PBmf50y)r?l=%;`CJ`vT7XKH7ktdVt@IW&utv~2> z%5GEqRQ=x14F>nx{(aUK@pBFdE{Ii5c3@cJM#w6g7JLtHsaRma2m*{Stq#7>6NbYz zun>5R0N1aGkD3Z-+an?2GO#q#hoNY7Mh@{DWL=gfsuTx-%5GxLo9hz(<}Q})NrE7f zX%wILR~IDN>;1tm%24xwL9sV<^G#@}Zny;R*oNTJqvGH}(}r6`(4j#3PGyS+%@i$b zB&(TH*Y4mi=yVfcMtCNDu$Y69|CUX%>l>Kw1N~{YDHyG!6E)%ioR8r5l_4j5Z+x?- z)eR28e!q&gX$W|e!^aSU1dA{U^3SCT;SFEzZ=!V?YOeVDEB|s5!dPSJX%V=}6v2Nr zCKt;!3W2vve79K`BGbtG5PM#U2qSfA9U%cdW!D1j+?nM}$bbgCiPDgvklW}LrSFi_ zJG0m2*_Zrzm{rItnZ!F}@2N^wk#*%=MB!DT0N&M9OT#;RL-5Kehwo7L2kXlBAgX%- zVyxjyLNhNkTO$6H(Dj5E5x7gq$DR{j5a}8hymg9%f?#2D|XFfW1lVztWEZbbdQ(E-5%Owv`nCH9;>i}DZCc?k&9>h`3 zaunq-d}|BM;DdQop8ro=!XA~Vof9ysI2^#zU02Aa+jtqojfHrqTog{H@&ZuOTC#Bz?m}|h8SZIHWf-EAKtOo7Z-{)vo#M1>QTP*4p(GqJih0bf z0;8IaZx{ikReb=F+3^1jABfusN~+tHW{PvQ^InEiIf9z@j&4QEhs0I&`%_CJbV=>_ z;ELLiTQR=7cLJ3aMv-(Ns6_ukqZk+Sw`n1`p$G9I{6Y;a@f_ibf44YFz&mRioh2f| zr(Z`z(y4&11hq4wH<6z>(zmNok!EA%>k-k-em#FpTs}S$WWtD(1~u3zgq`wl7xBrg zQ%v~FfG867YZOpKdPN%$z!5=10QNF-k)1Hohj1ctT$+MnzZ_W|JvIPv^qQ9TiIO^r zEcc-&6IG<-M7E4qk!y4X7RoOQIBI#oRcvwGT_8XJMZq=LWNb>lx$TfDUX!rb+uWm{nq*^ zQ7*XI2StzWWlZQE41T1lykG2dLU4~MYh4;9;;v8Hq8cCp&c}SmR6JGhz5cZ*it;3; zB~zbz%=gws{#>|awxbeXIxC7@_)G;fG6j+JLFtp9qSLrsym!eidI*JpPb|W4p9JQj$v@#r`Eo zJ}md3$*Juy4C7EtT^&akhb8L;KS=>}z#Vm_^d85=(rKK58JAC9=r8<7pv1pGHr_if zBpn7{9fK6z;&QFnMXWf9AY?~1@ADaVPF&%QKmHi_7zYedtH&Sh0zZ((%|%REXD@#7 z_H*oc1;D7Kf0fi+p zkU#*wW%^2l9QFKB9v9>>|NL9rzr@1)lfcgG)gU4Tk2nHpeoRni4&Blnfm8j zlve?upXX>-lx|U?8_b%i)O?tuac&c15#SA?i~FHPLj#TG4FD@hp8k87>ls-ClmYdw zu*4?d>4>Utp&}C*u^CAmG7REF_-FF)cYP6M%M!Ouo|XRn={%_;8_JL_f%pvI=04Tu ze*1>W`@SEu@#3HHI=J$limj8E{GE{;2+-yuI%kLQJ2aEez}@c+MrqPK)T8q$8{bI_ zocWcVFOVhUYO`cwnUl!C#<=7whORFv{?OU34d%%q`uV2VSJEc~af~~8Bq!j1#h-(H zj6N|-@3+jExzrQ@yvr=~cd(F+%BI}lV?nR`Y8m3-+Yv$BT1i|Z?(HAIBxTc}~9Me=_M$w~5 zGmxKOz_@~riwx=l&KH45nq7s|f$Pt>H^0Vv-@jIf(sC1b@c;MR?x(>f(pdH1Ao%x zukDshjv&Y^0r!$qV4)D6ft1KpqI+gp9zKagJRMdrxm%e{GCKiyJ$v6X^C^nS7<@EWh4rlxGkZ&tY@ZUU z7y-&t@JkUk^_ym&6aP4sS@WY+S${Xdz3o2D6IQL73+-@f`_L70{$h#rreWbj+ri^>qRW13Svps4&q3*1<>=jMck$#=l=H-FHH_|9;_kD(4#tX)n4%}`{LZW{KoH+SSzh> zxtnSb+=qt@J3dd@D~L$Kuel!?*FlkRv~&U(z%y@($U`lj|nQYtGTN#}4>IRNX6*nVgka@qh%s)b?EN>i8>w;yBt_wY=x97vty3@}k zAIAMJAB8}t4txk0bACssiKw zJ_=i?4^e{+pYG879)I<{u>Z7PcfPBAcLam*aV;FGO!C_c`GB|bsYYL!$BLl5wNT_~ z5d#~z=yv=}`-6+ZY;uc=^W`VJ_-o%pCQcJBMswShb}BwpQG{el(sb{Sgh_-%SA5z+ z#u<`wJ`{~9gl1reKs>)m(O5I=Ea|i6zheJ``STAh z`kQW!bVIH9dAoC+@n4ZNZ-h!f$P+U8q8Ut5#hYNpo(W3M-L4 zm+KId^0hpuD#`@)G!mpl`4@i}C4*Mt;6d^dhcQLWFNsQCDItmRXDIU$h0=HWH?EAi#3b;I|C6i*+ zk?~0LQr|9a&l61O(U(R_nTdrOhf0BmPx;J0E>8lZX%Tq_cCC2eCQ37(Uh_*W9ggW7 zsTL}y(sk)lV_bS*)=H1eiO(1XcIn>;EFs}GDkE!F3_*Vn)bC}i&8gl#&oZ|EPOxNj zRm%+Un%q*m#DDcR5?fp-h|27l)XjZX`QQE&lOrx=QS!I@^5=m{bY9jW4a<_@tpu_C z1ve?7TUqX)P?Z^wzSnx_ea<&1xSYUcUfSXBw_r+<{wI^^WIo}$vfflIWv}u{tTG#v zQjK#cK7ODq)>WtbI+h3Gz$v!Fne6#le|Wbjf#R7 zSL^`qe+EQ@{Yxz%BV5H2mC>RcW>)S@!qlIzlx7 zOZB4ad#qkJDQAqdT3ZArGIq&7f0cQrqNu#)0?jU^P2|DKhl%ZLkoiBFt49^-mM-uP zQ5nKLAlL|PvQ;spa0di_;z63>jlCs*+P*h#?iyEoSgY7ajB7H)-}`K|szU9}-ouN< zT}j@CsNRpIz5F4&pQ}Dkm4qs>@fY}UJK&`}PTbRKz_x@|7+{&`x^*%@qE;i0M*7$= z8>;aSL%KT?E>_Trh`m@R;$*6}rjDzh0t~kOLE^R7zRZt^fN%1s0eL8D4Gbv^f~4Zq z#aoVM4Yx7^x^j&ZOUr69ez`MTW8-LQxI_P5Q+c8_ts+*`{L>Yz8f$?Hj$dId$b!I# zjHp`%6WxTx=k({-LgpU-LpE37S}c}3d-2clvEgt~Uj_i6((&8L#biv$o{IV|(&;xD zLAz6Z%A@I3C2IXpkX=6lble~$;#V8XZ~axZWCU;tm@jVbRP%ek%qso7Ye(k1U=el4 z1%3uMx=nQsL?%5NmX1{y((;hX`nnUdXoT;70ImDy*8SB4)&n{JRO`d?2fo&0bl&O- zmktF-d70gMd7{Y%Wji7wE3fM17j5PU^m;{JkkGNz*q^@|Xlm*ee{%CWWK6qF`L|xz zds#ES5okLP~`khj;|la@|e|o_l5QMcK7PzPy3!5D_MT~TkA^% zAx3+B{NShnu1%JbhM4uUXk{M(2Y&vRH+AJ_rk3?b@VFczeS>BFIz#-1qt56XaZ9o< z-dSDsCMntG!#Da1P>c3OyfFoB@Bp(fnzoMNZT`hkAliILOS zS>Rhv6O2C?nr51ea@{3Ex zp9FK613g!sW=kS%DxyjeQLYLfUu)86SnGxIwvVy`xwegV^~iOHMxUBibccEVzzU;A zZY+wL8>_t(USM>k>bAF#e=^0ES#E;kgZf)cb5pXk9ycV!kcslXSjT|tcWx7|yl@(4tm;&YgcrY>o1K1(AZEMEfGH)Qg+ zllU5c)gbDSZ~$ioW<*^J{Ma5ID3y); zjj!JNWxT-W5eaQYZb5MO>6rgPjqCW)On&s>pdgI61@)kpp@L>)vFw6Pr3`CXET`>I z?o(MNP7{3#K6Q6}Q+M8EZ##fTA@G2NVareC955+_-B61P52e*^Ve&joCMn=7Bwx`V zz=$D0qy8ZzEDIe{bRhAhEG(tMsG8x$MPc3(pFW52A-(#PB8&>V$HLw-6+^%QQryD9 zR{<4HEpjQjbMQE}yYi_z8o|P4-yfsRRcpgq3KKMPz3hc6qH~iTAZE?0aBYEI*{7EH zi)rDGNTkk#H}#Aj#v4I09K7%x2u;Lygww)nAj8jLmQ4@o5litspZwx&g7?a-vljw- z!Uumv;1F09ak8X@S<3Sm_lF2p9S=p)k&UbzKH)XAU91mxtBM~)pgUiTj9&aSq!L%Z=PYsy^D!9egp}k+p>~~U2x!)9sHD_i zh!5}4qonJ%Kg*XmZaU8-@I36KWC2`C2Wnr^Brj63lo5&e0_G)m(axS9Pk=Ren19ke zvfrgZ{y-$%*~=T0)H{GYEN@N|lMr+c4woNAYI!UB9>SNikyp(yeA7}MeTbUam$g!Z zDumH?DmAEzG)SZ=Z=()#Z^59_PDaS-O;7Y@xD{rLqx*q^>QOZ!Qc?A^`Y_}= z(*yKaJy+wTde!8o!z+~G4T&nCF99)?UsNk*b z%S^e0!&>wCo%UBC(KkACf6G}draSi>!EW>ao4+$qO9FV#Tq|Ft9W~JcM9BAh{u;u+ zBvNZ-&zs2iU*e9hLoQ?VgET&C8#3~0teO9TPst%rs;RHsq;~Ladax!{_8%oal1lFH+<|u>Q zpxO0x!KU&Tg$bHaa8lCgzdB?TO0W*Sfae3LBLl(Ynt7~qb*kg9fza)*{-ef*^z$eF z%py4~QU7TYy7&Ua`b+Ut%jkif-hTVNR+r!of9$iOJO8w;Z&`IoJde}s4wNf0z(cJo zqUle=JQxz-YO<8o2j#r3ri$?Tjh4CvJ@ctyF>cnwbS+~7*C;C-?yk8Br{3(h0{3P8 z&JD2YcVoZ?qJEnZNr?y;Q`A1sJwZSN32?#*f+Rr-Pr5V~{CBlfb`-X~#u=Jj`s#q* zz|$h+-Y}~{8q&(o#YfXfm|K~nALM^ogHoQnq$v8M9sfQ?1gCdj(FVg)$I;9|q6)pi z1O|-W1%LHhAw4O1h|kGC7os7JM7Mn59qQUyLbqzM&gaGe%!K!_(zCfhG zKh|E+HiuD)k!jL9h*h_VS4}kJJ`W9b{nQlV*cy7!sX0U){QVg{qeKD?1Ak>H4g5nK z>%p$?mf%_P zMwM{60>MhkRN#Fb&`|XUaieaBwQ>u5OXs+Z{C*pHtM0f`s3;J<)4?9$=n)UBXHVQZBiNEm0RhS^fM(sx`*I8 z={Re?-n&8zB{vpH4}8iQYktU(naizGZ_1rR-Ib@4Y82vt;g6GtG2R5(K-4(xW zhRtOmaMY~yyI#5m{Ue0UPQCuNp6nNNc|SvCnI6A1Y|bYL%n5ioeYo$dvn6a2N{g(D z3hEd!6WV5A+M?tlrbq`K`8s5J7e(Kq!Q$R!g)P>$s}sgVZSl8ziZZ9%omwKSCwOo@ zAkJ(;G}6DcC9&FvcZlj99((_KR%8(Oxh2E4(3UC~Pc38G@>{D9_&ICK_}95bLCaHI zwZf@zpOf4lmA{^8Ad#+r64|22Ciyi6$;8#~9=mn&=Hu-4tna*wYqW-PyeBQG)z7iXO zG!Hu3W*pniz_Pxt4x5ufJ1$kN+*|*#A^I* z?10SVe2}`+MksfBYx1zi{Pi^cmToq}&UddAHuK?g&E~0I%Rgel0BOU^G2+ZMVdoIP zP)hvQIgAg#p24DJKJfEm zjJ&Nb{?fD{0@MPe4G1+UJU~}|Nz6rZsY1nW<$#*52QVTPppbKxkeWV4*P9hgFK%6u zcc@QYUvJi}8!lN(K=M$bhQ)quwCBTaGDmVP?xH7_%zZb1ua0=61INF=5OvRpi(T(l znY!J<^&;zz3(2X}1IFM-KtH?zc>K?hPY+M;j}L7Y*|)W4sSIkFOD}_cfn|xZ7Yr*O6Vr!60LXh`e$CB;@%o@6XFX0z%nlxcoH+aqhhW-CXCoi+Uv@ zcbMzbtJlU}N2Kl_zDs)Z@9KuV#UD07ca7nre**F4S7sG!-n;oj^j`I2sbAApefK_P zUM_^Jd(X9hhv!DdwEW$}{=GJJ=sK01yo}k()cDT&4+Vec7o<^54lbISODp`5K7tBq zhg)3r2|TY8i3;Uwwl}iV{tvJ2%JTkie|{X+*N9&&8zclP(K-|@>FZ;>5@R^)zDGi^ zSb6o!ce4%4Zz00Qh!VSBH)YA?ep?_Hz1A6%Wwxn03?P)l92u`{C(#FeHu*}L0nFtU za@K$xdfNw#VDEooxOTtqOl4Q$#J{|cl(gpQ4BJyX2&4m^sB_+I%N>Y@blRWW(nlYGi&*~n+F$az6%esPlmp4ADt`nyns_<`o$)0r3A#p1x0PU_N}cZ}=Ck{4JAA?K+*P-KU&%@yZ zuo2Z$_!u#I(f_z^Kq+fC(Bo|+PfR}|`}@Y~U-(W>?p?K8W+UAMZ(i9qA)iaqf_$WB z9WI7Q>8jA|DJ=FK8QOEm^cx4SKV{e*{DUK7Uoo=c7CR4>TxmXx|7H1Vt@n{L6f6oG zxsziWH>%u>;@*W$XN(qq-O>0W+k&CK;F7ptiP(pZuAH2G_})NUP(;4y=Octmy{vN0t* z(5|~NZNr$h=)9ZQpaNApAVeN>N@CX@%Q@mH=L%rtk!>|z#+o`)Rxdz?BSq%e;C86_ z_38V#;5;~zaS_IY`)C3S8c*5QJzLN#O+GmRFQ4ODO4IS&RZUHj@@@ckGz=Rs!=3sG zfFLMK@y2`t&R%h4x0~eped~7k<^J)s;hX3U3BZILHi{e$P$)i`@Z)Bg`gU+aE}Fgb zp!6m(MRz()R3gjxs3)@mXF5@ro8=|v{vz&cItdRdxQnGqOs(!JlMGJK?=zYdejMt? zh55i4@V!a3sZxgnkoY6m(GnEntD3Y4{P<)~r1(9>qmyc65?XK5bwW0vH9ma|)Cv9_h$?O4)C{{Yk&vT8<(0(ql%K;PEMwsbZAY^lB!$Pi45Xkd@_YMLLun1y7?x zj?mkhUx5D@Ci-Vu{wnlq!3Rc}cePY{ITaBinAUkR*p0x8S*LZ^`f}R;z;n`(MIB(W zZk=y4#7;ZSP<-Drj6tU6P^0*%xg^KpQ+Js^BiImL*ye6h1I-9gh&1}13A0^mjp5_8 zVlD$0|M5Ii#Y}`s*|nybdUm)?GxLKl1waA@HfL7T9kv;~&m0rW;1B#oPdalkF&#Zq zb~HCC>U@~_8@xYFT~TeO%S8JwyPan~^DR?9JH)d9|DU}FzZGVCgUea`NGcU$TBG@9 zA{WzHK^6hh1m=g>rMHazo9bQ6IX}~PLt^TeL(J!A4wH}^bOiIWJc0%3heGJ=nBW}{ z)Y-Qzm3TkDMl`eG-NoX|U*jSaem@_H<4vGf(-@JbOH+ng15VdI7?rC@-Luy?V^&uWmswe|_|T z%p`oybrzxPT6KUp*R5ht6Yv%}m?d}~H!ozeEx0(zlva8$uW#bCIp<&U| zSR8CEeglEH4A~mk@}h$4oYI`IP{V$(_`OafUqz;4pM>wDc>5GDp9?|Aet(Ei74G{) z((?KKU5*8B?r^SaBel~3m*6V*X!(L2x8XZu%Xg%ibOyz9`GJr-Pg?J7ZG|%>7G6rH zam(q&mxilkX|nf7iCEf(ZX7+ix^yKvbMn0e=|?1t$u3_UxAf3=HbZ{0m|nUAils+u z@$F-}jPSP`k7WvIU21b>Op)eP>B`TS?FD`Fi_Yvyr90=I?Thl?r@CO+h~+OwRSHF;HGjk?Sg_ z7I_-W+R68op^&2pRx3w=w{!A|plcn~dQ;eAl_=V&xVHWe2|mRED&1pQ)m1rvlS+E_ zh`O^ap$u0QEk+2k~3s3u^@CEm*ypwCkJx#TUDj22|D%2niaRs2Qu) z@B^~tLqA=!{(256yzyU=z72X`iwOFI#33OQyeAqS)-GC0$SGUP^B73bTWc+|Nu1SX z@&uG?bE;Z2j!a(LTdGp}EZNHt(%Mvi2=sKeMGL3D#cXAb;3z8kC%Dlui? zw+E8p)hp_+#)j#7Gg9o@UzAXAdu=%IZ1wWp>}9^`+uPE9-f(lTvQGQ!2%#I9z6wI5 zSD}n=&PHYv{bW#p(Jx+Lqm<$4_%gK7D|+uwPEQ)&-kJ#dp&< z)u5b%dDcJC_E>>H{O`?vk9PK|GqpFfj^p>ZoE|a_nqF>P%l=OrKrSHjbUxAtP6R8eZzH1MeMWc5A$L3}V8aodD-Nuk`}_TJes} z?d-o-yT0HMmcRkv-Dov7|i5fOl}m7VJcGdgLb{EgFH*zL{evQx_%GalO2Cy)Ie^28LA@!dh#IJwZ$nNzgF|}i#gkM97(wK#OJiBU7#x4=>$r=>z544ui;t9y4FvlkiPrF z9d6ccuw|3XTra^5_;dBQ?3TXV#tVxg)BJ;V`NL<0ja*npU(Svi=Dkl!0AVMEE3$~ z;^ljLa(nxU2w`$pxI#n>q^61Xb~)IO3`-K)YYk8vGxCifB3 ztXB-tHE5mi97p9YK-`JKz+#_dr|3k8y;tVoeMz51$oKvJ!*a#89ZH3Bfcv!b1IWsF zS#b;qy(sNMD)2jZ{KU5YIS&Fznuk|g$q*O*qd*2R4tkLN3h?AA{JR9D?|M*>Q7uXj z#$xY0*$0DC;uIh31B}dwwnpPQI0T4zkQ%7L>+%1|M|TkyLNYpxhU|l~>A*Y8Xt}k7N{3k{ zaQ|SD9rlZ?vK?XXLmuFMxG2dIq!sC0jEPzt6DFNc! z&gV9y$|jDC1mnr?ZojwJAL+?FM{%G|>%#6|;*TD={NUI*t{7kQ_KQBA8aUo9M!iEE zWj`7{Lq~J%qfu;%|2tYN$bQ`kYqq?ww}&6rj=m9DU;%=~&&09zJ=gCs0s>RU0iDK9 zd$-3zNtOPDyTXYt$6DtzA{ZvQeFAZU{T(9oN5#5^Gza?$`K=1A$sn|eth2a)>vnIh zx;oMCil=d@-$i&J6r32B`ibqU?s0H!n8_|A>poZGWP+wuL_Wo{W{mU{QNAtH(<`LT3B}-~s>z!P$aCFcbjL zUrJ>2N>pvF8d~vaFT+q<-M^o66wd5iIUIP(9v;t4GRp)6UM)>W8jH;7PXi$wy?$08 z4lLcf>-ty&y;H3a`55=1b*?h9$d188w50Z$d4_6QGByeCX|A6jNlhw3T=T_msM867 z3vEKeI9vikI{OMXv5!e`hyS|s0axtPAGQp{Iu{*$(Uv3*E)a^-e;3=bnomr{xhF{4 zB7(5(`b>3#k!vy8sR;1N*#M8{4Sgk}#|PpNg}gju2}wopiEx;^Ri|l~Q0B@7QZ_@= zat%_O<5%&a;>Ogt)591VNSz@YQ7}QQaM0Wb$T@X?q5hQl`{^DNH4M%kthzLaLo?fU zgrK{0k}l8-A7Gq!BA2FFb zVp!i3qwTGL&Fo*_S`@JJzsiH7JgJina`Kd$$;>$SN(%kS5B&U1~qem^g-pWmovY!i`%pF?0WyWUOUZyxL)hq`18y4nrfZBe&61{f6x0Z+xNKh^Y#9v z`5tcHQ-<$nm)lx7=0!5jwXd=E^DX00<3`fKR^yL18JIg|dzUO<+WSK5vhQ<^&)=5% zeIPwtH0Qf?bCE1(t#QzMUz+3k&UvEuW|HA3eWLG`_H~i`Yw2codzSKluXVcK|Ik{q z(r?N*(0Vi3=ux)bYtJ9L-fwodEUZ?v}``h1km=i1*vdV2m%pGUpFYq7VNedVwDd0To-yFY5ZrEKx6 z{qA)~A4`opYrbK7f2V!ybX}U?JeGZPPiM`2)bpeKbgDTNn~wXP-dW35*i3d$o8lM5 z_DHt)kbRnO<=h{#=Rz`lY7FteZu?r+xP11eH7}J@a_v{m`Kb3~tJK3qpBJ*1d@cPVZc#Z^aZjgItlHFS6&k>}A~?n(v)-zsNVv zl5x@e<)Gi!H#yU9)|yu~avo94W?VgJFXM_O=NauKe|st$+ER<5EPbvd?@;?z z{Y;sp)AZM==A#bpE5@vsjr8zUzR~xX_L=D2wVo}se&)`f7wvQ}Nhdc=Qe9KZQ`~1u zWxd?>sv^MAuMB6;n@YNiH2SUxQFcv>4%-~;>?C=}{AhUGyoahthGy5ly+>7XgpzTI z3^cFmoYtM`vphTXGSo9=)0FW=vQ0Jr9$&Z25c; zMti9z#;xQ_?_o5Rb{@5LH`4zkszRQk(1ofi$lG?CR&280)aN9sicxJhT612JpYOi3 zy&jwVWJ(aBP>1dC8Ad_)r0t6>$MxJ*Z==s3jFI;FSw2D6k#RGUe3CKuc4|6l@_Nlx z#dO+#t#?*of6q>}H$2shV7o=ww}s;N7= zQy4wwvt?B7(|70pjr1_np1!mW=x-ZFNckN*zm|+fYOl@C*mKZqaE{8;pfNg*S<^o# zX&cUQ#7@Q>9pqGFC(_47x~AHy8+P41X#P{UPOzrYIQ}QJ?Xl;I4cEm>6cQXpbuE*U zCFT7Qg~nHHol`hxrfqVr|Nci7RTWIB>A^M?ebx-P@vL1@Ilwx<@dA18m|R#WSO2AT zkKybHRF76RQQagMbS|As&vcxr6UC9Ai;c=PPjOdY>9KO2RTvML-5fSQz&F4CnH8+Z zV^qB){)o$cn3ZyU5wm7?5JnPa6TA*q7->(d(n-oiC7|cQFApWJd&8F~tR+U-IYT|9 zvwmlpV33zlJs`MEOcN}NeO(Nrvn%}yRggRa(*V;bedX9mR88JW56k8=vbz}zvNZaJ zjS}nYxWGSWQJ7^B#ip=Vaukd$wuC(#);QvpobLW)s`;iBZ)QW`1;$Yoeq6DgbH8Zc zFHtoC+|t;a^ux|LDxRFMylH*iKyfR3FPL0k&9_9>4)KCG$huxhC!CkS_ zrzo6+cfW))XO{(j(Gk0iVszy3&H`dz(von1pdxs;hu`G<&>lZqRLB~6C-Z>3#d7#g zXDOy3pK(63Y&v+8>{K(jbg?Qa>!!&B%Onp44Zs{Z<2I(ol}q^5sBT=I3%lh`mad!V z+%*q;4QCFY$y_QD-PvxV_~SuTG?S62JhLbUii%0~R7BuM?qA^8@exIy*)}>dIzD)R zB3{TiZHsS+J0&8`{*Y~m+EvAsQO%|Ha@X~LU%%RWpa~G$L%1g~UoW)iK8ET~FZCLQ zL?`8ARAJkn&X){HPt@WJ#?Ki?2#tBkfJPmhm+mTNjVj3vhq7tT#d*?D`zQZ!{)68S zqtF6qTBsQAJY`XzX7r#=9mJn<;`1HRKQ6 zewBPRiu<$DWm~(SW6>AOM)a4Qu^3_ZamWngfDe#2bl&8=Eac{D`V~e29VOVOrxok& z-#IU4-|rdkey+II9GRfqf-I1$@KKS+nGjna4^L&T-$6Pd|Hy*Pu(eAy*JU zT45D;J$z?os6HtzJo_%pXSzq@#BpEV*ZVzJGgzB#6h8^qdTAOVJHSs+RfMU?w)vfR zCQU{-+2`^ypYV>g#;&a=?{L3MhCoJe?xe|jskQHVI*`pDQCW#7`mA-0x4)wLTFx<) zp*aCe9ZH{_Gk|?_ueYps$&}6_^sn$DjrZ(lVa~wfC(;weVK_SBs23$OeVeOeac=8Bdjw>gb@)Y;jf`6}H8T%3HD ze+Rk3p|hUFsW$L2tEk*EmrtGk4_*WSO_b}(f-MwT8-1U8w-%KX-5c`u z=hw^K3%A#9-=M$nj;(3H#@DT%TTlHHJKop9?Xz1SWE!<6_P_Wnz?&%!(YvT0(VKZo z^e&hh_2scomCAUG(;Jyo@yu8F<@j~6nI0bbL@1odTLcC>P@U?(3Q5nJ6f%U2W4fE>%$N3EJ1iLjpD`YbA z@oT7u9E@-59`W7sUvNIa=Nj@X_Dt>u_AbOt@Vv0;eb#>)*gerKV7Md8 zEo}C%<80L@(Mq|`$c#iZuJp4a_RLLIqWc0_R88V{LB=}cgl8*D zQyH5+Qd=7^j8we++r-U7{?_i?^KY^GfzKw|0}eKEn9w3up#Q?R$>iJzY(4RxMN?ov zrUR#SuGMG{j|>~u^-1V9v15~LiS!$Ajsf3VJJw%Y>R^Na13WA37weOCI@AHb#JHti z>JUe)}P`Ck$S1Ay^m)23#&R9Kj`*35&jEw2ryGfhoALIqS1?9!CW zrl2KhX4#=pt$8e+C8&s8UK|OcE3>`REI{x8vF1^VLaP)jqu%%P{_^|_-zW9cGCS&& zcc}_jvjS$Iz+ffSK1>6+CV?@UQ}Cw|+AcldRZSL~B=d}EqzToL?^jDZ;$#b?P;K7C zW7A4OgT*fI4Z7R`Sz4uzziVt*x`X9>G1IA{VA`#>PprPP9rC~%OVz@V4&fX)qQ{{ z8KjV`i$rgFx@lOret1s5a0qVzDDXX$oOWz(Ruog)rm~tJecjS*v)C9}XiH)>R&E)> zvbjSCK`qHB1HiL!?gimW9{NAy`NN~1c^MghCJ}sF5EfcI*5AsK0iS= zK=hnI;PXefKc&v{-L|qJk@h2F7kl$BH{@3unh(S%$Da2hJ~z2Lzji7r--d>2R=yL& zMP>Ph7H-BvC6qkA@tK!~Vt-7Xt8_WXatHB|Io|E#4;!n~na>YqeHm4Y(Z2ssH6kUI zLk7;oENXmK%?*tf(B9|-=;mjp_G*oQ$}@C?LBCPH7LQ%OOMk|7%my1rSdE&`m>;*2 zEvE?tr9p_;q^&=()E14|tPRoThjb5{NPXe>-SDHcB0hs;^JaZ46oiEK2vdfzv7Tj3 z;LeFe2H-A(M9FqK<84at`H(!o9lo(dFVgj;p_Tw~40CN(AOhX25!WF`oBczw+h{S> zs`9#IDie9k`=8L^X}#St3TDCMu6!taDfwxGQE{*_?g1oSs2a@$?L?66KFo;_7qn8c#j zpv4()A&q3|cZz=d-@g%+QAz($>%fX#q$-zI>f^lwG4|71)`aL^&hRp)0;kKy2-JWD zUU0!YP>L>6;FV)Lm_9D;;7Vz6S|PvsV6P_i6PSDCq->YoT&LD`$a^@}nS@H!bc?ya zrKmmUNhEg??sGBFSF+-f9ZhlVnY`KVd|B?+FNh;!uA?501B+WC=Ti0C zf0`_*_<4AY2gOn2wr*+oz?la+8zW^`EBz|r6}GnYX+FCuu~&Wz!cXHW)T*}S5CJ5C zeS#hHb<#ub@;!#d7wjHZ8>h=1cCKy4>V5iRXp3;u`QMM;H(L3}kq5sWo9YnCnb!c| zzR)!{?bHLHLyO3JssD)faq8nW=V=O(BXG>KE25ZJQ{YOQ>b*{J8(!fq0i(cw>zP;H3s`jAf)iHStDU+y;V*IX2fM`D zyC{OTiqAD0Gw1s`v^p$wHL$5K_|&3(6ng86+?vWm6J?!{-%fG1&EiVl?iOQrLarR; z{*B4+1!@}0CJh-xbnf`IKZF=2&Wm*JICmTFW_3XWQ1Mr-yt;0lx+O%(M}b<|2ZxjsV2s zaE6VYs4cU?d1TWifF0XS`f@mZ?PmnDl-AKDPhUtUxB`{wb`+aOMNwooYi#Cb5oZoehkP%w&VtsJ0`uFdb%TH zMc@*Fc0F##0brXo{V&hb(A($ty<7Gha%(4KoaFi`lOn}CIaQ%#U>=&QEgh%S@r<;E zwnFIo6kK(|PwizU!OWqxgHKBzR_ENsweqLAU2#g>b+ zMO0lwg0F2gl4nez&hFiX0A=Esb2UmPWNc9uhvrY2FRo%rhi z`#P=9B_yy*Uk`kqRf3nbsZ7VDt`vq8nNxW!|7hm_dd-9e)j*Q ze!_V(a$w$*&zsFb;<-?8+i8N6Qt&D;RE`byIu`ziSPd&jd@B{v7yIILZzf7Fj7BhR z8rVwOVv@?H*$nKoTsHWIeLx|?x&Xk2=f4oHL>oS!m;-7nDQTY|os3t&!+!wC8;bt+ zlv6nFnVqeog_gKL1HI6`155rLc&?$CF`xGvCS=@u2;tsCw^`s96kWL;*(YYiuX8`o_i7{s4JANVgq__+k|_cQMxjcL`J1@%1CUQ)1`p%t zbwoy|ao)!x%pRU{RN%K>U$kM~6;Zw=FJBq#Yw3)J1 zNjK!AiJwf{0?*m2{zmCw>nFzqT}O{ztc|^EUD|dhrmhfE&6gy#`)7Fj5<%mL%<_pZ z*cL)o&IMkyvbY+-YA}UlIYrR16F1qFYj^1j9LOPWr$y)ozo51`7j6Z4Xon8L% z%L_%zQN@?jlEp568P~F87qoW0D+Rq(8%RaRUArcW6;ysJ^-scIiS{uF)>X7uW4(_3 znGH81Wpqt+U5wRmAo)-n`gYlS$3~aj`PD;sM1tJ|q>gX-oc&847&&!qTofX8=!A?E z)Ro{^^DcPd@CH&ij8-tZ{e_EXo8HS5Lje!MJB`lLF8|a|iSrU%`7>Qxl`D&n0D zgR!-!SAH&X{wyf~6Ntui|?2KuwW`O+W9IgO*?El3-qH8rLq#XKgWbQ(_@xQD^f z@P|?qlNGm-5`HG=HBvS0rxQS)>`5SkZzw*N&@@bW+dL^l`vKkps!04r3C*hABM9W+Z@lNKMQ-oBmuZAVoXaaW@B85}@I0NU%@i7yXwj;ps{ zU5RTTHS^ktARE2UYt+BO;Mc_Sr?plXScUWV<165Wp{(+19g0n{pU|6l7UbZIG^qqR z%NEsVIo1)DQgg@5%i?C9Ar40Hf}1;5H+S-1uqTews#RN*npZq(3BCeuzd0@JN*z@e zL5D2FaSv%x0_V$(oABiR&>g20+UH|nKFOK7dIYv>y0iS2TILR8k6_Ss>e@9x+TFPS zEHU69S^{cT_>^65V;BgFGfi$`ClB=K>GMwE!Na=vl0HvKJsS@p)RieS&;8yhv6ON8 zVPLM_>avh4YQDw4RaE{Wv9m*M!=(brJ28XfIP+{3rv>s)YgtysSL3f4e(q9!9E3~3 z#7|9^oV6D^=A9bTH6}DZlR4V z#o`L55%XL=isTdlwO%T2QJ2Zw{JAUH_>Y+TCA9_J^*tQTslwd`e63`&fG#unAB}lF z^-(Oglg)*D{!Eq*j3kzx4(pZsWo2I+e5JTwf*xrK-ei7q3O$u(bwMoSV-@(0Le3GK zPYs2}41UD>zDP#I70ZwgLI%na>BRpu)7^M+m83p{UiMoFM?mrNw?y5~MvHO48qB32j z#kT74o)gjb1B$69)%u1IflvBTX`OcW3c{t6F3TEdDIA+*@&c!O;`oM7rwg~G8RpdQ zgjCQ{1+x60_WqlibTOneM^Zq$iG1Fm9_=>T1gwQ=UEsKC&2|Z-V{bxc%%lqc$ZxgB z0wQM|1RcPc76ZR+f~0<0$0n5*0^t~3GLb3x^CkB9U=G|B<+7iz*|SgB$m-#kSQeG^iVM^aZu1!SuyD%1olQ-{eFBWI{r`RhmG|U-s|=UKhxN?R=p7x{q(Y;Fy=Mf zwB(oFZRs!;WNV8w?{Cyx@1;+(&mSf%B!T)U;30<~L&x#jbt>@#y-yqiYz~w{BaIA!^74urso=vBLZ!Jp{eMpiJDZ)R0P@5Ji0S%g)Om$-6Xsj zH7B;DuG`6(z(D`eqVDv9I)n6mvZz^_7wC|+JTv_w@rT0`Z~tvd^fz(m!SXUhTO z5nr9$dlm0PE^ST@?$IB}-kJ8^Pi1_)Wl$YY&@X&&cXxMpcXvOyyB*vef(N(Y5Zv80 z5F`-XA-KD{z5Ji|%Y8oFTU#|-JKeS0D?Ky4-M?;%x(`Exw0w`es18Boo;%OlwHiVP z0{cjbpHtc&L;g>Tn&B{0-(`%R3zLS*d=vH;)LBE*gS&b_Pe70jKOV<@cIN6u`t|~e zU5B8qKAEEvUDjWIb)7)Fn^nHsc+b<@Jc6x)i8-KEu>rG_AKZ7TOI|X^I)Bo?-n=BE z$g7E}*X!^JhYvsY#L@zVOhNQd*UXFD^^7<|Dfp!&PeNnu2NOsfgqjDs8f556=Y14I zHfG79$fG9LTBYCpR~~)yQ6_6qvaU_O+}QWl1qx{+?oyUDPO$aqsR4c9dpid!CArV_ z9+|7g2q5+V5MDy~2aYBHQ1v-=-fiMoWi_5%{$ z+<)$VKmgz|0EU^HG4$(bLmW|2XqkxQ`hqmHE!yOX}qonL#ge23QsF{;Z zy)S6AQk&zY;b#?a;p4FG>6AaJFA((j`WD#Z{`mIpop&X8@O}}PdodOJ+OoI49ccJ` z5$DAGczFEYyZt)z-n+A=@hW0_rz>Fg{`fAuyQcN%Pvq$*vmL_Vc^wkycXY<`VN5);Vm+-MzaJX z*h6-T;QH|7^exaT_aEJ0PS4{s|8?)ud44M{rqSW_x;uN>#`Npd#pAj0=j&t8#|O`P zkZDQk6;?&7?2}=2YuF%J*4Z@^b}A>zgb1%Z=*9mNTxNDSQ)p@WpzAIRX*K6(2*y%n z5#7YT*OJvbOwjWOpbmz)`Q<96d*dk(B^T@ldD}YVzTU*hoKt9k<10eYTZbIO|8Nzp zkH&im^LekL38Zx6HZ-PjRB-#xK6K=J(kVgUbz<{Fc=^hfCCr{~!%}(x^zwN`Swexp zw-Fw4YorO&nM3PXyA|%|eD^zzS_VO;4hHkB8M!)Ra;+p`MzyZE7+^O{tL^ ziJ!BXmTGT@sz(m(B-8tON6XvVDcIo^*~IqN6Kyv>6Feh3Fn)In62`_yr$CD{p8WCq zQ)=_E!2WoP>G_57W77Rk@eAThrz$q=GkzpQ|3*7?z4vZ@0Pj(>mBzn~t&#p5At#|L5BxW|N&jR+*oi?l2 z$k&qXbu89Xg*F?yZZ^sFu)-2jtyOob6 zt78uhB5x+vg3f|$uX8-9NQ%#uN3X$p0oArO6|8BzeRfW`;_$9}6S&k51Dl+V6_^{TWZNNmvwA-Aef@W9RuPrbOmdRSR z5R!u`gL*YC)P<5ar)JnTX-W2byPSz9-kK$0^v^h>(~JJVRB9Z018FZy_;o@0!3zx?}rlO(~Vd&!T-5B@wd72FlyB|}XGpEIet|N|?PkDGim83wM$E;M=0~R=z)49!YJIbD-S?8(Jdv1% zSy1_2q-DpRb?#1ksj}b09!bAIu4kD(1^RhUhqD+uHp}zv{Bh{BbhLVT?X3Z*mY~u_ z4hO|Dpv!vR?%1iETfji!M_>Eeh$UFzn_EahSsFvX*9BX|HeLwdF1_gURpp>rkrP4N znJl{4B?^0dQ!rs%uaLJepY5$?+^A%L?-OWHwG@G@(BKh!D$3p1%#G+` zC3%BbKW^}Ov|L=6BR)cM#_PuCK;SIHS2LYh%pRHKgkwL!_$k)`m_&>9dsI$nhx{K& z*OY$<$}=^ezReQtKJJBQPLp3X2EBHM*MQUB?oQcH_50!cnaUP^xk{abbex?>y*D*C zfQL2pL zCdng&>0972W=ijgl^ErNR3VN>VH@kqxR>^u8S3O??RCPB_K(@Q67PK`J*>D)7DYH0To$#`&{6#YjD|>XmG|1W1#!faSslv!w5M%=I9M?@p zQtOujLW+WkKO8gavb6D~K2r?%J+I+U0tZ+m|_Ht;;Uj>)X%MKyo3fEERuc zf%e7nC2qr29dc`CX{dQA2~Ad~5$7yW9n!HHR7fg$fT;XYb@TTh^)c(ID#Yd&Hci;iC?jRE^4DOk-WsfmD-<_E$&0beG!o__?K zpx26#Sv|63xm_8dz*PwwO*b~J+Q-Ithm($U>c3e4m4SuvFALeBqGQxL)^ZAQ9wxuY z^3JV2L#eOml#Siuz{GgdM5WNKP^Qx4rjlOW)pAu?xacYMOgypjY)mS}jNv4-?0yeo z!e_pOkLDA($YqLDEUDwtOaF3ZfJsn<1AM_ir3vxaU*EN#Gw-JPrb1O)2&w-48!P8H zKhLIz@(-EpHC4y;23e;p4~^3syjPo6&|Mos5#^G}bkR-uNvvC>GNLC_#F_me_F(da zo6~;ClJRHUBG2O33W~~MxB63=@4nZyl@wo2$5JVKSE#LB0FsLQnTO?zcqzQ8`}3578S{~7RoaApMvGaf@*1z4!dr2QXTB%Ix84krq`)| zNL<6t_YE%wwS=^o7Tit{D=lEw>PYe&y-!Mg!l_%GmhJk3PPhVj2;f;x38rw%sR!~F z6R6vAOkiaQ@U|yf%SwxV>v}z7Ny@!&)aSlsoE%`}up6ryfqVHv5Xso6-;9P(i;DvK%2I)vE6)#9sJ^G{MQ_@Q3Q4c6QRF^qc0>^D=fP=$D z2NKe4;z(RgOhuOFN)?exZDST#e{{5$yf6NwUw?lRV?I6|w<&YE@7CZ5p=gCBTPqXQ z>PXnB%EP7){DYt(tfTzXHO@k7$}@(0GqLy2)nw4qna0I5qd69mErSYEQLZz+^5@z1 zLRr$%-`Dc;-MyG8Z9}lDMk+1(;)QovrbQ^F_##Ki9}9oH1;BiYelIAViLDB zYwny=uEF{5Qc*e>F3RG$`td_f-x9Uc4IIa*W}ioQ?igJ@wf zV}RG*nQfshXf3r{vsx8ZodVZv?PT(0z_aQ)UG5z4%xQ>YIBGM%7hM3IG%prTHXjvL z@k6WbWVMA9ST2{}n{<*N##f|b)F44O@EO0_uD%0jd< zJH(_bgoZjX76o(CUM!oR>~0||U+`OvSfmc~O0Me3K9W&m=$o!`#Okl&0=EgNT4k9w z*S_?vfWi9cKaW?*KQL$_Scp{@P9;h{$>E+TK=X~a+kcF>1 zImFXYuY`P>ZKfBkR(h14QQm^+hGVbqIsMY7X;7pk>#*)zNon$u#8XjD6k2HwA_$)Y zmT1DB#|@rpnE39m=SAz8l)PV+KWgb0s0zmth_~mW*+!aX(Ve4Dw(V7UfVup!t;(IA zW7jUK>glK>QO0kaelQDSdV5%lYKeUjw$D~uX|s$rBlh`GzKnauomZ{R)RUdCII0pyf%y8pX|=9e3K2q z@0VaoYj7RcTm-6S!f-H;3z&lKBSruEZBh?wFBc=#GmS{Tl&*J9Yf={13@~5&R9dVJ zJaRvj8UmnskrzGHejd=&1%N6GrYSbk+dD16dPr=>z%-dB({~C#_?ee1#`rAh@N>21nxK5_?iF6~G+*WyDk@uwPwDL78+GYh?j-w}=gNzKV>r9X zA?;t$`!bAf;)Z{W#5?A})+bV#3%&HQwQmx%Dl2Td$_h_49K%t17zr)`zy8k~9+9OeeoCV^BY4Y9Z4&y@;bi zszzlr(62dw$40+`|6y)5=7GH1lKM!+H3obeJl6BERA{e~NpdY*e|0mptKb3#MV zIGwMYYF}o#bc9ltHM1_9l@?fTX?k(&y?Ub6nRn079gyc_s$3apY*o=B*nzZTLlK<2 zz5*Tm!?liT@)AP&h0-*=Y2m3r%Bc2PhQZzdc+;HFXqkB_B6$wIF1`XR>Lc8R|1q6B-^yO0v}$YW9ZSD7Fs&_8_~1{gRcc4%@KKD}@w2w221TYa zVtSLFGo-8~=Ay#kA_yg%g=gsCs@<6a1mHTm0)%Hu0Fmwa^n!B)C$?!;~-1lfMBHdk}Km4DQm?*x~09>g%Q4U>Xq*RQpWL-+bRth6&WXZ`-on0C&o?jFEZjZK$=y4gdTe(}T!qqY+N z^=e+~qi0k24n3a}1K2cwsE+v{a^Nwa|Eu)O&!2ZD^-3w{L%Of2(-&9V!u3Kjv;={^ z_|u`MoIHh~+oKuju5?3CBv(eW(i#(ozFq9x#|w^AQORHpk;Hq-R;IddsRRA9=qL zqIKy)&WSoM-A{{}Ol^W(V0s7erSV?){7`3HZq4ciXRt}C-LXs+A5|Sa+Y_bvz0F}t z;+<$ye*59FkocGu&zYZc#rIf~XQ0;PD4>o(ZmINQKBfr?how&{F2O*2>A|nP<{b1Y z1K%=hw#3=p^zL?|53Cedmu}UHDx7RQesq78*#u22p*2{-c$CP?2WtvJvy*RU!p1N2X*oCN9@V+y>u>ba!OjjgiyRXwA;8yn8;8S^dl;CJ%UzOv`0rI&Za z&|5DhXJx>(t|M|IS^0j>fX8Aow%%JYEQ!m5noGd}w8<_pdRcBUYg59ym~ywjT6FEi$u3;zr!VsFr2BgLTwx7NHjIbb z`08o`0K*>z>n2Aefeblw;t86j?Qu`#WzDBJs}a@dD?KGMHgKJ*Ez^0g9-Jjga>qkz z@9G_eq2yQJbf@Z68^Bhy1QP9b-XrEZ2+chNM1$r)MN_Ba(T}C>_ zCM*u`V;OX|tTNx>oD!qr1qts-xE?qhZQWJ+W~GU*y#;Lh@@DL&s~+|}}fT3nn@i>=V=un`HX zBG3{)(~PUfqOq}+jvZHxvZqnh zSD%jgd*|4;pnLuM63Z^T9!PxCjNE=sS+k~{1wT!J2W+Z_nmA?^$(4E3FwRHF2-{oh6HqP3{Beh^5CXi& zn7#hdxe3K@EsQr|(v}Q3+E9-5G;EDCJRb+aL;9ijU_*3iA=$|E`Vv}9{<}-+LNY(r z5Cz2PJj7%yHtz?V$GUZ5IoO?~pZ2!cpLH1e*3Raga;i*2HQMxquRFEgoM$DYs6p*}&(&!MGwyYD_Z8kGI-s!s+Rc+8h7;i~rD-ZTWH? zBvXDX>Q`B<5&y9bhqYWr$1J-eM65{}73k}_N+iqehqsHv@vXW7^o_x&t1A&Fk~NcM zqM85nCtD*ycZ583Rd8@Q22uZzpL=+d$I zBCNyEzsucq^r!(UZHuM>&M-?S`g@MtNuq2RmNY%bO(T!(YpmJQQK{iAJ-HF7cxqYv z1Z>+@C!s1#u7fmxx&P(<fkLb}mKV2%*3jaMaG4)X?jE8uO`$!(XjYCEJO`U z*@G7x7+OH~($VktQujS7XA?g*#v)~oDc1|F^KZ;R8V08XF!?2=7g$NIZg2OjlYAdH z2!onuWOTLp#^J&6Y*B77;g6P;7Y+Zbz!-Vgn3A0E z#f{^;4`dT4sUKP`K{={C5E92&$zoZcJ=i3hNHH4O*xsykJzqdFqhGzmZ*G~G5WxcM z-o0Qo{X9Si58W8Henq1ISj zq~rg#X4Bi;=O^uOSYiyAuNG1qL9U5v?H`j%Gj#Ib9F6KdRZMp$$=zK&bc-SLt?Ix~ zOM#{+7O_Xt6;jrTH>E0K*QKF4Jl@OwugX&5b8I@jU;x87HswK4RH^e8FCk(As!-JT zMDWBY^!_M%F7&c3L0GtAQMr`72nlD%RDDv}6mr8Xhu*R%TAyUx$EAP$%*au1z;SySBIkl}05*VCFZLK6m1xSkf#uykj5g8WxtsO zs5Vq~wsx=h%zn0;{`CIZJrT24ICzC;7L_}y#Tk1YQ8w&8D4+WO>p5xW{m2J_p#T8z zQ0Ne_S7@wX-(T{UnRPIL%qSiuN1TxG9-Iq(37Z`p!2cg<|9?ni@iIB$-GnTkZ19S& zQ?zUW>*mq`*)O{K!T-?V;#JA=_cYM7tEo z0QLWqN#;K$QqRGfTNubphISFWNfp;r1OK4SvQv1Mc!2nS;%)C(1cQ;ee2FJ7Zg2(- zZ6zl+ydi#;mL`3XdC?0H{NhPI{GVGui{}ZIe=~4^hNd>cP7|+A(`8nE2H<}&pzrON z1w)WoeM$Adk7aTI)c?`|Lc{}1t0+cL7#Y^0RX5k zMP9nP#M-?jz7krRJf3CL5IP0tJAs;Je4pm~^%n{3Z4P1J?3QfhgZT6wmvMlO^n-Bx zWDc8XoqK+4cHok-|6y?Vop%y>S$Go*k2aC{(lYJ+&;b@z$u8G{P%`1i-|3Nq6s z;|vAd`{35w>m<%7;_Ct&<<9NUCY(WtWq)OEcTzF=ghXi ztkjPWr$m6zvZJK{ZM^TX$Ann3_6CYx`Fa|Kgk+=79Dr@qoyH2{8A5UH3AaLy9WGuO zy^L+Y&@-;{4 zo(YB_}aLuff4=jYkrz=n2hyd7{>Nl|h zCa72fi&avixw^twkPDCfgM&!mG<8?Ackpk@9SsRXy@N#`(Jtda{8UoG)4CSBpfVHK zVe7M}LB|o-8!MaW7I{R5SG%dQ-e)7~A<4v;EAh8M=95;QzoZt)q{(rjd}3!bf2hJK zbTUp#XpZbsr@FVUhX7Kwn!4SXM5;(&&gpxOfIki8N}Mm!aF=r#3&`%HqJ_)_2gE$` zckSLem{0A`Kjl;lNB9B)t(?PyHXb14>O{jhazCbwo7DCf+k789g;ZukvoHRpNuOH4 z$Ey*N$Y*WQBpR3s*5R&i-{q;QYdP)k+yIDS!*Fl-T^=#oea+&UC2lTlBslS0UfFI4 zgQ?7M766LSv&%Nb*gNV7S0UQI_3#WTxt`8fNu>@+L-jrF_C2NV5*z0L485`MG>qyE*HDbBVpKts~p_UabT>4q&5n&4l) zTXgQ8Y)HUp`PxaXlueU1)kJrlQJ4*wXm^UZIXrZv*G=z1zhPk%4addI7A_71sxrGKrv1ffn9XPow!Q|+XH;Wa3z>jOxd=Pb<2 zhxI){hy7gm#@{2Jd8+9|R$o`vlpA$rUVgG?!rBRn5WWmndk#r!JvSaDbbUJAjq_ue z1e4-4s^xM78?_O6O&0^w78C`i-2ep>NWVZT@6WW4KexM#@?r!}-alJDCkYJst(o(t zzVSogc3vvfT{-?QhDaOXWobl}bVVZe5<0oXI`LJlN0465*e1hp0@Zyt0^d$9i2Ijt~2gAC*>#GVzqX5ULO&OJc zW~xPYgVYop9EFi_27k?1A_<0i*-$!=1#eW(cNn+mlcdsPSBdfFpG_5N9QKggEKJH@ zD8ma~bv(wr_KuA7jYN*$BIielGnraX&01l~4{vi2?sBCbPbH^_FV-78Uu1W4drXa< z{5dz&X{h5sd!uppmZ|7F?+GW7zssZ4hZV~^*$}>M{xc4ZC~Z`Oc(h)h2_ihdAy)+Y zQjP4@OE@KJ`DPT7>SRroo*q%?XPaQrt2KO-R2JPi>YHwrTaEY5Hsc>FsyPK;3!=Re zrn*~SwLz+1@4%2#gpkdGB5zW8MViCHL_{f(SE_LfnoK9X<|v^s>2Kp3#b|~Ced{Vd zFh{7p#{b}P2v37uy&;#!aY8p+To8$OJ3`V~QB@lBLQiub#XC?PxTR(e) z+k4_N3f?MX%$oxa(tanl_i08p&Nb69u7L)NqqF?bK~Cs z71Y;s78_f!kvJEEF`j~&HDmm&1&m}k{*1&3kZpYa9(+G8cqm3^uvP#di`=$M6npP~ z$y42rZyiRDu9v)dyjP)zn$`l#Ob=G6G5gjgS5*w@+J4lzV>^gUyzo~p*HQcRmy03J zhnO)$!|*q-8IuL0XMfxu@?^iD6kCz3MUDmokhT=UPUd}}UT&;~&jk(~*3SHUjzXfV z$+y4>v$J#|K5NTP2W&_CNRBD_N(m*28O>vcc)8(2stG|&8B2f6c!OkrO}0mh%n}D` z2EhxkyE>HP0V!SaG&IhUg5g5JD;72tmi>F85h)LM=<|K%j2huv+8w7~oKNp10 z5wnvdHJ+ZzB~~y!F{kx&#Y*M&CDVI3zy?~`_#ztCvDZ^tWHa@vQ|a1>Ci5aeF8m5SocS8UZ; zhQSx0eu~@rAgM88{i0DKa2}+OKu0h)X2LD`C)YUhH`dP)tI zwBX9PJ1t2G9%4ClLZ=hPVTX#H{NM{iSpudJpG$Alcc^~DKZWhXkf(C~s9;rm*Q^Vd zLg{cZs?UNp#EHB8{hL~gNCXO{#Tj>q*)dbY>ppR;0ePTQ!f&EHtXE?$EoP}J$u-4F z@*eAIs24kzJq02X6>AAQUSCpeb!XBLm<4}|j0G3a#Tk?1jacAu_ml>S2D?5asj%OE zRwr`efss=R5`u{z{2ap%;$qW#kSYd&{~Gw>OgAN}RU5gl*0TX@n>M-xoSY6PWr^de zy#s4)RDpJ1M~zzGCnc%jz__VqsuoWuDXZUldnN@J*@@eHZN6iZmlML4SD1)|N}Cy= zY{h5eT0%2Cs9S%AbS&t4XB|RwMZ`&@hny;JGX#YQ#*H}X#EL-qZHikkbXV`!&=Sg( zhk>4S@3sF7SfM)G!X<^DYBPX1=E-Xq5JluX)9k}xlMF2B+oufB>4h2mT*I&TJdxNb^Gf2e?wR0BDBiRfKTK09u1fuQ zbIDEzRA~^T#mG$G&jh#gV+Txx^=Br9%(@L;>@d0k?l1G=lUCvP01gEzt6Gi=KjO#f34-p>x z=Psh_+fSHaVi9dUlxlzq=bDGb$BXLN-y?y>?+$Fu7u|v^!v2(EF?DjxvBh|}ehni3 za1fcpmAT-Skc}9mK+k4As&n?d% z!F8ffEal#m*a1NYMdxy|V228fB&AeJB@AX>Nw~yZlhANWVqIbxcZgaK?6Sf|Sd-?R z<&?54n1}*b^JLz<7fL28?stX-!7ieds6UC3{;a_cv$pY?eb}MUEMwuZmZ>1N$*V|} z!(nnP;R(qWL*NI*55FoQwq$6*w52&3R|)7&DMDfv;5Z z;{shc2wgNOW78F?e(rD#E#qNqIlEv}+e={wEy07U8FdHxE+3-Vh}mMnH8)6+$|yJ= zj{TD3Ad-M!%Ym~%53f|5ooRypv8N99rhw=H|3RaZKzSx?x0D1;UW||S9j3$aiYZW= zzOU5iW@H?Z!8(L0?u>(U@f63qn5>V8?t74(y7ZR+;g}y@4~2u(;M@n*_11&}>9Zr? zHVVvV0$+^bo$y}a88Xf!e-G&GXkulx$wVlhvY{r`=rO05U7M1~JQ z3;4!59`{C6-WiM5Wz=lsH!)zk0FBG!4LM~&C6TDMw1jHJQj-wyr#U4^@s@~Gx}Qg& zh%bulRjvjmY#@)6ZN43*2Tuywm*UpPNm|v8He8u!md;+~I5m-JFgj6;p9>9LN1@35 zT_lzKg{R+|^Jghx5d;#7qZ0)|JkXPa)3<~O)=SlrpF>`+ysDN}(WTab^(Q}=fgL=6Jo%W}~ zof(=6JKHvt;%_|g=w)T^2q-1q00v=PPY{fd0v4S=wkRf$it|wzt<8y<4^jpt1n~g0 z1GlMG)VnY4+x&%}FA%WdcN>=CaZNz~Ji8CTWhcER=GSr#P+s<13WGt@?S#1L5E@ewt;ny~eR@gmv0t+H337XUaKJOMRAk4-vQ$4TnI z8NI@%*?pcOIsnEeV1Oy*ELVmxRN*9nW)Xf2QrX$mZEUbEHp#T5Xh>?bU3wDq_X3dkJ6dtc&ye8 zG6tkZeI&@OagMfAgZwZiV{$i{XS9upLDox_t$L&=Y}W3-Q$e~mdk$u5K)>KO4(^#b z4vVYvY*WayQZhjFJIud{ISv{Pgn-inX)HMULmz#7PsdCra9_EXayM#;ANJoiz--OR zZ+%Yej@^>-a}2a=rO^m5ym>Dn5KOhUM!i1KQzA}an$GmmXHpcA8@GK;gGZ!vIgST$ z#Rz=Z6=6h)j$!Cs*+<8CTO6ZB4+3L!Rj!cqFBOwvnf0>l$lnWaZoeJr+A0_axk~5@ z6N)doZIg1+R5S0()j)CUHU4^|YUVCC=2=@c;uS#?BOcpD0N$pMTvGLsRzu<8LFeO&XTYACH?6?ZDnEpgOJ>5-ypx*- zh4-&Z*U$)p&nE~AQ_q^)iTj)}IChQlSieEQ$2;TbxZY0_f*z7AB(K2?QV~QXz2#~N z!SwUJky*f~SBu}Z-@g9H+%n%-?(KyTSGwA^+c;~VQr{_hFfk5RJIi_zb0gFQw}hA}Hy>)ZH}@oc z@d`QuG?caFciD^>*EAYmYeA0vq#S{ph_w~pTo28raE>?;p_>94OcwKrA@yzJ`ta|4#qxWT?^ zv>OsI3KtB2>#H2k%)e8A7aUQekL%I9RT73}qvOF8fYRdL04ho!U9tqI{?!GG2X9qQ z9ip)-&$M>^Dz=D`GOO+fSDNH1-+-++K(@@8WZJ3!SKVI3^?S1i*A=XRYZ}$-H=u(iS&F_pMNn@p7)5f1S=O#O03+CN-~R5IoMnY!X9w8AN=?KzfgiW{wTTAV_ zHO!o$&WJLPI_T8Rb&`<+Ok?~Ew0Gz)ev1Z9(yFZY6=1V2cgS*YFjoPMS(WR_%Jk?U z8+j&ek`nq;%Dm7B)3SU@4cX^Phh~XRvoKaFFyyZRnMp*hz`zeohGJJ_;vAMws)O-T zkeL|@4vJ=`xI6?Y{8n{69^U7%3~oEC7*+iw-V@SRz_4iuT-3XTV&}T2m|E&IJJl9YI3?>n(s4R#pqL6i2nA1BP@au&xsnS1j67p>Cfc= zorZJ}V-2WlM%nLBH>c!b)ei#wugwvBB`KS6BgvPr*w(L7@_k%Fd) zUL&_Tx|7c(`*MFdFq&nv$l>mZ2Uq>`DP>$zfGWwhQ%qmx1MSpuT9wa|2)_F$LWAdHitYb3&gO%ewP!sOB}4@EO3><;0e+oowuM8yfRe<>&uP+ zc&U?lN$d7|OtW9(3vnHYfJM!~-!%Ot7MoJgscndp8HuL0(iZiE?lk*KT+WmssApop zV!~K!zGPg)bOmh*i@Vi>Ouvdl1jRP240fp{N?%xFyH^|T5j4G^V9xG}=~iOP-hjX4 zYU5mtc&^VwMMd=>@G|xZse|2N)AZNE(ZAOnMYtuK5#y$m8w%qP0-J?cja|or=}9q* zx#nLRN;XM^5`s~MJtlpJp_6eQ#JGovF?eB~!HE!1>6Givz!4+cJT7G#!gSD!T9(cp z?S&_rk=Y2!E9^eS*49Z|FZTIa?o@|g=5;FxC-=vRuGo@2}sjyPWYeA|es_ z4jpS(asyITMwJ$owwr}NX@Np~g^`=!#-wBcK?E};YBo>Gz>s!T0E#^?E0sT|PCr{` zgO9_h>>t74UjZyf#R;-*DebVXjM1vF8Zjuyzo02vu%!6X3o7x9OiIigSQY^TT%$qd zzuuX!Y5EuyaVD5%e`d+v4T@3;6l`di_KouDkN2Rx^k{k9{}JOyf6YMX&)nd@ z&X;H4r1(qcR`X_atnu)lcCySP*C6^Fdf=Tr4ag~4;L)WYR2fdg-(vby#?fx3+Xyot zD3k7kBdRN#zwqkPo4A^_Z7W{TN5G-TQzH5ERrW~t(F-myjlAE=ZrSK-%bn#*S zS|h92cOr#SNb$y!5#!D^9MN2z+@dZhXyfkgywPLB(D4Ol%*Pb@WktijuMfY4enkCg z73gSf=Zv@9)nJwEv1?Re63?3%CMrgroV56H-(YJZgt3YzJHoXYu4Ex-lK>V7*A59T3ZMl5O*XI&|A+JRf2 zqR`2)Z&FxUHUP=`FhvMTc$EaT5wG;=$qm5`Nl94UBQq6EE0Up%iua8u1K#X$;)0op zTP2(c`2cRYn4OY|trEsCd^)1iUN^$pmS#xGxes7!m{(+%2QKPz#jOZ}km0-!85(~7 zpiQ{k#$o)5ef+R`$GnN^&+oXTlk4Cd9KmH4c0)N9!WB`K-fN_ZE%Qj!c2Q6wN4(t+ zc2}V@v)-VN2>Bo?cl`w3_4mpDdiVKM$jqsx#C#{EmP9a22%8mN_M6wBs{G?hFZvyY z=^`5=Nl}5eM757{iR_Tgm5|fcvHk^$G&h>VjNuKEqCcLKguc(M3&h1;rFI^8TWud8 zZux;;`rM0eb<|);(=0BBTO{gbUi}PEGzw4{q^;)og_VUaA(>|X?h>XL*X#v0DWii- z+J>TaO0C1c!j1%vSp>s?H9rKc!+R=XfCf?u>!Xk!0nEA0P#5Ack}XhVR17K`j#wUEB-wy)pT?yKwwov~hAlMyU4w-A z-BLZk;LBVhq%MM7^Yv9LOAbBMW)WCx8Bp<;2PT{oVzDGl?=U%I#O>MOun?I%qA_)F z8-{$F|B}iOw{ZNovZYoj&h;z%8-`CT7N>6D+HI9BvS@&p@`VrCn1?gX4#4RPE40=e z;Kh$&jgy2HrZB0&D8{>RrGX(Ubv1lz0~k+{O?uRnu`{GRW7+UGa^sSXIq1I*T#=Au z8(EfoZFrYg%Jb4YQLtbuhRV)w+U5_IZuR=MkXY9Qqi- z8K@e!+W0EWN}4}USh0l&F=1Tzhr56K_RWy_D&Gob?97jcd*5L^`v_N80?e9lp@qA< zJzf7=Gu}eSTM}YcRutpLx0iDVqt95Z?621-?YSXW_$|+H?6A0fd$~&xG!pbp12Wc9HH>Nqh#&cpF-!D*SAZRN=qJzb9e?&4g73`L3_ma% z5+XHG10>P8dYqMluI`M}bGn4w3$h&0K{7%6J`RCxCQ#xj&l_~i39bf*q(`htozjN> z8813*wP8MmEeASWdg9>pVu5nGF%@LU1bjo_MDB!6~;Od;)sDJw#CyqeJy>I(}-B zlRvDJ>J{RBKbn2|2QK9l&{z;8YJXVeRlm>??4h7bHe)$+2$37Ui0xAyHKp+r)TR>b zyU@s8ADdebS-t=rW5Of}B?4h?;j3I8UZBWYo)Kjxdbw3$dVEYqYg;{O@Fke1&)SeA z_ne(r@3yF5tc!V&;z{+ctSP3!<=9`$Fuw--`1U{xv@w{R| zH_icE)Pkmx-;DfOxcLxQQ7GYjpjr6wd5Q?%(mgm=v5hDugk@2-YUI?i$Tw7t_eXr| zl#A)mKZx;-ZaGB=^tU)J7p=YA!CibVdXot|Ut#@jVGj+an`PnXiGMjJ%K8!S&(UJ0 zdK|k&X)7o%R4~dcBSTQY_0t&l^Tj5^7qaqh#2_ItQleFwK!NilIkiGJd>D5@#P>8gX)>Gr$si`h?q$2pai%nlJLS z31EVw`lrRJ{imfQU1$h;(TCzHm=OFXMw3L?der@Jc8q8!cpNV@3}W(+>E0(H<{%&)j2c-R-m65^Cro0`=41`+`i&634M3bCw~WuEvr28dmox zWfOdp7iF-eL~5lIr0_otV%c|t#WA=O$`E8?CB{7Uq7F5}w{Ic>8k5OT@wJI9jonvz)T+ps#b7I@JZBHf>+qP|E$Czl6iEU@cwrz9A z#{0Z=s!rAUc2-r_`UBmyYIWcD)v(6sPQgh=uT=gH6CV6_OWfo5yw#j0)SPbuIUOGM zzZ?3=i$+yl=>Gc~%~GH_RqiYz1_zxw>h9i{!jHLY-#!#VC+11WC~ONUi@yOm0_oa> z=ol+l6RJ(=F4%b7aDf42?dGWP-XK<~4kt-ba-3?k?Wb8}6e2CiC?g4uqgKD02H z{CFQb_VX4Ut(iV8#|jDg2$i{)B6*n)08^IN$uQQ>eXW8cU?h=o@aPkX7`KP6@rqq+ zrV-n{#rdaTgxs_V)}XhQ6B(FmN9vPFL<71!*vNoyf$;*5+0SvkTkH>RndA+6Xclj` zK}lwUPl4Q(|LMoDX``fqxcjhaX~GGB7^fW7>tXoDW^+x&3B)z?7==yb<8qA_(Lo;ngwQNbOGHC1B0`UW&{=i%WRvdV#eZeuI1c_j z#YUqUVk2;E7a0F7PrEy@E&{-!B8CsU7uQ<6!JSuawkt!aw)|C0>M{Q|rvE;~iI*n% z#RJ>z<^q-ZWUp@3CZ`_mX7XyGq}iCuro`*M4m0@j!j2WD$rx!nOo;z8L{#8=0 zowxPB&gVC*x{Q9Bi_^H1Fqo~yq^2xtT8Rlo4MZeIkRG?pyLGf5}b4AjDtv&_eflR^^k78)3a!{p2XD#^WGSmPWY5@g_Q!@t^?%`}%%{{{DXd zFFyZ^ufT7=?|7R32L02UDkzAH?Tu#bi>WuptqyZ54Ltq*D~}|eS16#eDhz6Wuc`a7 zB_iyc(R1>Nx#>@ZB}i0$8Z8f)Q-B}+y&rGW7NMBa7fS?wC_!F;3vd&tN+gke2EnuPmfLF?# z>;Xi?qZdkwRLQxjrya2Oy|w~$d!{?D9v`GEv@Wqny5=ohl_7o+V-hs zMMKWb2JM5l_1>FeDmR2cEosO07gJiR`gXZ&2tB_Doyy^c##wmvt=*lsTWp21#zg-B ziA1GNwML`s%njnhQmR|{ZaB(kW~Ho^V6T7VFIW@1RXv4(!0z0h#7}s$V{ncC>2 zw<-(~-#MaZOUCqV$Im4j0)c?Knbhw7mi@;%@Ht8m2nWC>a^uAblazur6F_)s%xJS@R{=0?Hv0!7EYsQbzd3I_DERk(^!EyfCqq?32YdT-6o#H;95DiWl~ zLye{R}EhUz0n#StLZlp6E92uNj#vvV6HPUk^PcC zMMAQpQ;Btg6W*R3oTl^8C;=oOmn2{32;fH{*)P|??#@~V{^2WGeI}4nT1ICwtv>o zz~;%m3lm{e6dxGaI>B)LWy0u&A7lxKhvOiMd~!0>AyAVSwVroaHe06NtcuG63+%Dl zuNK3L5I6Sd8NL74x;>h>>f`pa9D0`yklzBEq3oX%lZ!-6=U^A{e||+RUqQi(vn^Hus@3q5o>` z7o8xnVNcSdc-vd@OjpgbS4bM3mOLInJY!WI-2 zw^w_%%}20t<;Zx+QJ8rK><7NKi<_58lZ5&8j`~kNO%cQzC@o=MZ`Ac!h)O0LaH;x> zwzhL^T<9COy)!?f4e#t|UE*kPC zA`L)bVsbjO~rUq~hNGdi4l1b&L9QF-Ne(`GIgM=wa zx5oip+(p3yv>31lYgxFI7jhXf7oJf`jB(mC?vAQbXTKDohMv`XWW^*aM(?u^n@w%F z_(e*V5NHCWkGtDy}AgeK9~MA^`dg-TA|d) z_)l>*7?p=bAz2d1z9W-C^iTa|qApSnVqLqIl(WdEr-UDKT5vM=B5AnRcW|x9aEl?fsDpWwIB72H zg;z9GuTH;tUo(F{TJD%ZmADRY_VwxU?e5+BA79}B*-1mQ+gq$ii_QA6OC(nI@lg0D9q1y8Uu=SbCM_}Wjm`=j%g zpnO|OKv{X$lxF%bPHcF4EYm_0zE~V)pnaG+s~iB6_C3FU(>1EZC}khNFSp;7FKW2m zt0d3{%WRREy##l-IDcv3>NfzVO1m`LW~;nRC7la}O8p(64qTD;=B69^a}z#>1CO)Q z=YeWYG~w6e@^uatXKI|k9kfPYHhgeRA{6umSONn|CB z*tb|HrctciM+Gxm$X&U#q>0Ev<84tIPd-G(+v{6!2yj=hKF{k2@7>72@%!WrDwR*{!kVzQHFx@>7=;e3etddH-eIT|PEcYzD>4EC)m+ty$v zsP12?22`4COHIz%eWY>a_-zFQZ#O|`R{H~EoA~U=QdcsDqA5&>R=V!INak!gpW#KT z6fh6iPfw3hg~YuaH#Su8Ny*&QR--I4O5PuA!@pCLBN7&5i8Ky)R8ZYtbRGRy2@pcl z1y)1E{OT;x9BU}0QXg;=T~3{SX%n7kYK*1+EtX#dUhUsBvVJVB{n4*kz`Z)Y!_VDY z{8@Tq@JSK9k6tnsfXNq?J?++>=pg|8hhX%iu(WJyIcN7ya|haeBQ>1J9E=!8bbdH0 z92+=Bvq9km#)V7;ktFiuvx+29UARXe%?4!3+l+OVf|*5BPAq&u>Plr(NmKBC56V+Y zh!y71cmH5_#IJjOjIy2jl^hD^#z&HJf+nJ}uoC~R5hHgj(%?Y&+AUsKa#B>vY|1)N z`d3dPxS{*CxgzYK2+GR@4}4<0GLGjca8I0FC$c)av6tQ<&s3+%CulkiDZq!+-g@ir znOETqdw&UH^(fE&hDXLSNmh8LB%&W@6LaQb>}cW2C>K$K9U&7xt2-yW8s+(m=ZH7> z4c>Vf;Sa=zu2kd`C$rEb!<%pvVTZybjQR<$LIdi=BM%-DvOH{&=3Ca*q{D-5c_MbI zJjK5H#G|JowFesB>8ZtCCv{%&6%0^Ub=DY?(8EEzKNtZKbXsGikjJwaY(YxGvBF;S z74m&y>_lS6Fw6pWy2653n|Y|?u5huP-b9Clp(v*lLV-zh^*UtVgg9PW@R+PfCaD@b zI?g*W5_!nla18tv9Nsmnv_XdBRb{^)<&GmlKc{x>@(Lz1E=bkJh!3pVpb!im9qDQC7TAuL;U=nB{4X z1prbjbTG$|=2;ixu*2BiC640^wzojC7ZAD z%4@WUxashMLPOvcL+SS3R3?0V8I9e)V-D#Qy!ymzS_Uu!t}M+uzH~hEZlXJD)}hm zN44!zq5@i?|FISs2Px6a+}Wczg6o4%cGF3j3g6{dK1w|A6_CGN6SnM9U}#vmap7l* z!%m3hfbi=2K>}b2!)Aej^qQZ^>Bs7SU=J?ppD>aUJgmjLC16j(B9v5o*Z&UIlZvtMQWufnG6RqM z^WzWL;>gKR(CDwQ=S(SL>imC4SqzrbCfjY*U`3H?gI2q)Rm%<<)(ekLX@i@kAbB2# z^OTz}8?QPhGpGY7SJTP6a$O(HFqe@7?r0CW+!O|tB?_;G>$fxdmiF?9b}3NWI8?*2 zGHVEKhso2|@AyCedYP3fWtfNAdO6Y_4-J){@XM;Q7Jw2ENi^YlhFvY3wAy>fEMhn! zd6#O{sx1irWU$k9T>f78R28FTzq9|8Rg^#KMwBcGGaDOywhb(2Zaqw?ifo6W2mi~3 z3!0P03Y&{8$sbszM{jtSn4rxXR1@oVYK{X4@c!Ce{js*vhJ8XpS+ZN+seHY9`lWj- zyhZ%6EsWd@MQ)7|ORb)4JCRVFCeWA;A2l3{PNnh3x1+_;fS@Y4%EPdIi?~cw{q*CD zg-lkMi0{%KTaggE^I?0y%h^gI!qf*LGz+{cKD@f5JnuhpZxvl`GR|};WLG@b?iCVQ zu3IdLdtm2m(YE*x{iA~+l^2kR4+6n1-ed=}(;0XK@lrZ#n|B0+^pe=u>!%coZxE>o z%MQsNitN9TxoY^#J$}9+{oe~dDn3I9&kg80oY(Jx{Im{Fy{fK)zV}V6aBo>nHZ`_z= zbpE^SP1`RgFSu?&og^LuYiXi+=oF`Q*~-TP6m0fX!v%X=7QSi;oQy1`PT-y=l<04# z!&}3Tgw^8ofvV_EyvO4|R<2;fiNC9S|KcLkENre5jPBPYSIXNY_f-4ADOe;ZHl0x5 zx7_Z+XLwv_7p5Wp)=2w%ODK9H+2U>flu)uphDpdoUBV9azg2oT<#LH@rhXbN$ggX^ zSdw~8iV`q(ze1Ed;GQZMg<_h!x2>4GnXF9_LSsH`1S|YKU_+Zc;dCqie6S0-wUm`Q za(DL+btZ#Md42!Q+|R*19Y?0I(=KYdxq?kWc{P8&C)?r^AEMo?u07goo`7#oTj?~Z zzRJ0t%e1QdkoKDF<3EC4GR^kO9N}1uVn?0rt#MAXe5{^6+&S~J37FLCax0;p?H>1=F??o`K=&~Jr#qp#vW+er1p!q`%50_pp2wk?j*!DV0z zxwtEH;4?fw9B;Pb({L|?6%K86FNEGDVsVVEUv+x}(-~c(QaCpr>akKpbMlnu%8~&8 zEIoPrD=C=_q%ROO!5r!q?Cd**5t@RkohLTQiluXilRB1(gKbhvt~vhP_|labFOzq# zS!K~zX4&Amq^VB@8%n)9WWJO(_oZb%7TeC=u>;O4JTrQeTc@=>ca^TpL#-MIwaU^@ z+g1~#!7kanu3j|H9_+}s^5q2`Ht`RAb!O&CJ%CILJ9TckW4Liqwwksz!d7xYdrM6N zmruC9${Hlox_ar%F{09a#q|;_T*k#Z!?bDRsG%ETar3aJ`a^eDX{~Pl6c0Uabk>+Wl;=+6~||KGD>n z!2|Yh8A-6rh+osPHz@&p8+xJ?ryr2=kP!`V<|i>l%l`;n3qNz`3Y-2k(w%rU=NRXK zJ`2Mqxc}Qv4-b&tw3AymuC8Df-cS&7kG)WVT6f`Qr2R2)Ql^Y*$h{hsSleW@9uWtg z*jeF#v`)Hfd-|h9RKvL`=(jor{Zwjb0v_Ta7vpj>kw9jx;7dN)jHuML3?`u_7Hkdj0$VRN zp9{D%%f_6{f1C|+ux!CPBV;d@xi`PPKQ|l`p@f+gH>`vB(am60cOz_p;1INSNsTJi zy1MBn>y1@57rQ#O9=4a#gjAwtgj9S3S-()$hD3adgQ-==7PlEH>xL+lwh+(_;ysRC z@`#n8GCPW=v-(V3&uQk6`o5F)rwWk%L(=`9`~O;xJ0QWXQVLuM#Ly&h;BevKaPj{O zTtZejr*ThR7#J7`h+qhCa2R>@AhDpo1%LlL1z8|$_>w#5kl>Jrvy;3JNgJ`1>*?vo z2SrdZCj(t|WNKcxf5tpXwd>v@-xozGAm{z=^WHHR)CBS!Y6hNuC}h$;>)ZZ&n00e8|dm{q2X|7!HC~;kW%P%!yEc`TfBJR5Br6=mW9_HnHFP zTI_gZtE?H!eg|u{z%&`m$36k}`n2n}8GO#Bvcke$68Iel@+G`FVu;5&qyDU@0`Pf; zIF75EZUb)8+xr01np;e<-89nncV{5Xd&wP{1iv-6Wv6RS z^}{k{wm#l7ocn=zzo*bcpsm&a!xiiQJ>MK(5dVL!>L6UOtE@B4v=yx3ouj7ND7EMU zm?T=Z^jXPj3u}3X7<}6tar^z>s{bN*6A412?&nBta-5_IMSH=aq;hDw493)dqN~9Y z+7O~q)%WSx&rz)|o;=rJeFq2Z2drJ>JR|vfJ-I&dtuzFDy%Kups5i)KjL~aTRhp%J ze2*t>gnrXZDnBDyt5cT?eT4yQSBt;aU=5p;-VbcP`4x6*NfO=ilbTwra_*bfA(s^* z%&fQ4cPjivB(BOwnznyVHKzZSQI1&;9>dr4{&kf?^RRVZz_Nl`H%A`0B6&Zgo9v^sdx& z9K)_=IL=Ys%C~mt&f~B>1e|j5CH(3FA@fxai|~b8@5?Kasg5s05D`cOZ#`55uLKa z{pdL5qyY#xGR_`E&oykS1uZ^bt+v1E#`~(>SI=47u!zp|)ES5@>r%f@Dt>swZ1TuS z^vvOF%6v5nn|uvT*%| zon?DhNeHrpM0zVwM(zP<1|Rc%n;a0=Z1X~O17FO?h5etQcbc04biLCFyC8aR$9l(k z+UnL!P&42r*DtqGmp!427nZprB%6Idf9j+DrBT2;v0IfL%YR`kgZpMlY}gQ1MOx50 zJQkqn91Ikgd;jA1_e{`yM}9!0*-@-oMs~9eAbx)-*ztZ4kpW62%J{Q$3M=~AT!qOn z>}IDAE(}+ae6rmbtK58cJh7SfU65YFMF2d}^0i*5hNnhC!YEPB{(s#-tH=w~O6|GB(S{PeT$mH~2_d>c1XO7w!9oP<@q8kO4~ zw}!sHY$hd1kFL+_f*KP1IzTGFm0ByLP9nl>xIgAjJ4dNs_=&t}$N0$|A zJ*WUWs=&})8J67dW!v4qTdMw`XcGRaL`KzI6N}z&q6AwY7Q?HMAEuz}0ZE_fp7VJi zSj|kbOy{a4`;$IIt-xrlU9}@WdLYZb=CCk$4s7IT_Ib|t>rH`zGY|Obg;_51232GV z4dm#6?d^Oi-rJFyjai%&{)&?*NqDZpJc`NEq)!bULf;D6A~x%wo@i#nE;9TAeqWBnI$1Q8Siio z?D8Y7qxL%B$&gGN99U>M1{~N0>iJpc1=q^&w@170&^TD+yaE0?GTm@{v_5Q(@Ufbu z6!Ey*XWPlazVfE<{-$?SM}TIYD{9`iY zl@Eb+c%9VfJMWcLhN{~HhSQINPS@R{R}QPT;pIeQA6J80HL%BZflOYuSSPxHm*Z&D zlk^`us?O}5y~QHV4plGNC%a^r-fa2@_><|J6O&giRg?SLFw=$n6S1HQW@3WBZc9%$F)Z>Xf+|%TqR^ zpswBSi>)I<7Dwvo8VTzXy&)DuukWRJvI9W~g2$ z;0gKXTxl+_nKd^+h?g*+*xf$HC0h;56XUEYkx`Vz+~}N8k-8m&^RS;E8?eHR>^7bh z#Hu&FIuHDD{wUM+0jnA2lpDRM5_2pukJ-+`f?CC2>Nn$fZ(|e_*}gJ-j|QImuBooJ zV2|?L_s;{JTScX3%6baV{D%B2uI(bPB{rLtmn)EY@%kM$hhjWVhJX*aaKLzZXl*B_2wb4Z@YmiAAj8S3P^*;ZT{>~ae6v(M$`s+zTKAGR zM*aJj)4b+MDm?WLN*}0n4>8uYhvU79G%zC(R_QFi>QRaLmQYIYPTP^-K)wv)d|<}0 z7qJm_ z_({-dmQXnA_=^kY{kYFi`O;d<02#crlkd4ZYbC>Xqw?@dYv1|DJ8(DmFW1+f>Go|Z z?K`dy-_2r=pwF+UorIqm$Vb4t%&+1nKG`pki0>NJR{O}E1`Qj7bC_m^ww-B+8KJQW z)`#)1mC&-*@u(B!{fyA=MUrK%k44;f$6J(pzE{!D{{W{u%}|fEWT=4TNXSQGYvC}J zYsaK6QK8eJ_Dw9I#y)GX=ZNdL9n_g=O4NbXQb=zev`&6C*bVm~mBXC@CYCkxKNziX z)jE#A592)Jd{`fqgHNMXLjS;w`^iur+iE9E`My&BoteMWT`_q9|C$UCPK>BdwxJ)q z5L5!h{rs=qA>BVZUji}O>Mla27MUch-uF*TPg9U<#IHQL_8S1s-GeyP_(BIIXPp?I z+5IO;FFg5e?m-!~K^!8-RDBY}6%tc;t%q<=u7ft4-QnjfLp@7<`lQbQ_)VIOKGcO| zs{H;+0bY?j({H)$P{h4p@ogrX8(?O_PQZ{bQ*o)>t|0e6s!BQ%CEsOxBoYzJ0L|<2 z&$(J65G)C=AyAX}K-dM$Kq35uAdzNiH*}}HX-84cmhgGHK(_UB+Tvo&cg^8E-1=ha zdf;==uOh&<@lbzXAFPJ|u4l++|4DOsN5t^%&}3|Q9Pt7jg#7e5h|yW))EtW`Q@IE{ z>C@!98PGf43j0>*HIB}9FQlry`R@NUOhk#Qu%{0jfC!PnbbF%Xt zNDG}1UfdEIT{!b=4a0E-Bc|QmD&ju**+)HQw0)5CNondOLlphVaqH-R9viuuU)+{; zd%1z}jdI{G2q#>|=w+Sf9oYq+M}Y^(8)ou{FlPLL#t#(MMpoHYR1OHU>c`Qrua#17*AU9}e3 z;*`D z&?^D=6koI@3OctyKheSUtIzq6r66{XdY8Sc;d-2O+K%lajgk{@boDl7EX{<+=?#32yb>6$)SV&JRcZQ=9o((Kbo|mTw~J^ccwQd z6bm>cx6!0v*XL4TM!7s)+0xby-yTnh_UY_|Lt*38i4PW2I1>3Zc!c(_g#@t0|YGso7ul+CF zV;FM^DR$RdXLQ;svFYg1?m0CP;x_M#uaaY?0=;wpWApuRq63X)iU75W=bQ6+Jddqp ztX6@Zp9;wYISP-JpY;CdH|A}yz*ELn>5S~I1iF;$M_c5~A^tRve*-$|WlzDW48-uN zetxP}YQUpF7kENdOE`s3R8NJUy24J%#PgLOUzZZ);XQNS4OWD#O(FwPi5>M8F)pPkqxkCI*@J;- zB-xmGe0QnK?qc7-DhB<6YbB%T_#DTd&q8bg8_ZQ|h_>js_*cXa6N<{7^T%^{gQ9k+ z%)3`rdi+J^G$Rga!;u&>1%n973Pm>b8{KIxf>;8GLv)IBbH1;~JQ1VPhqmeU1}bse zCc2MglJwcSoBQXiZ|@f{Zs9p#{}!!=;}Nul$cWlXkPD}3ID~NBdD#k=1d101i5vh!F*Ysb+!(eB)Pj-`(1@b z4r}=I@G)Z%V)jZ;X-4r%VO693?&oq3iBNq=#whO*KE1JRDZZwwNDCX^fc6|Zvh&vv zV3m3LBh52UCqU8=4MT}*N}aa>bqU}>o5tQ|ysQy8<}bzd@*xZNOMO2M&2Ryr1cYNko>25 z;sc0u@b0U@{5!4XkE5~HzloPjB`!L-fe30=@=?FpKt|2Le^U$%Z%?6B>jHK0VAI- zMkSrAfAo0U4SdRklx7g#6nWU$(oh@~4K>k*!DN%0a%I$_R$~0Y;u3 z*~2JdypIG_Nc6DCDXvwnFS7%hkdx3zS!H%teB^fq&V>8LJu*SW5qi^8&s$kGl$Sj1 zYHn!KQbkaf%>G7hc!Nn;pTc~ouOovnF8DKPba2i_*L~(Tk<9^-6w36OBc}nU46^Y&LacZdS^4Z!<28XcBCgda$N{Ei)~k|Av4Gk=)Vy= zlAk?N0^R%OfS=&HTwg8$_G8br%v$v!yO?GrH^0_ylUd?Y9Y=lRi0x%<|2QJz4blvG zKsx0cXB&Mj%%)-VORsU)8udD(@V#FUSMBf1S)#D#EZ3fGcmMt99`PltXKb>92t26^ksbv2x#ZZVTx}i*Z*y-?jgm8APPFK1kS6wE z8R_tQrFqKw9~nsyk2<&UiVdFLA9Y`_f(sDm-Kt{rU~&S^gJ%1D9N3bR@HTv>Frs5OfCpSSGSrp*)c5u9s%UEhhE!-m6Gm{h# z0SAz(h3HX8^-4y!nWhgb2P~-D{;%H64Ir6qY4tx4KTMt1yAQSiv*c7xl?c>9D=Yu0 zLvvIfv@QWsi?|K`{H6zVAp$ph5n0f)Ip^V|d*%O)5L9#jBduLV-a(~EOt4CfQNfNM zu;as??R`#s4<_Nl+cx_&g)1KI)A@p<4Sz@p!3VnwCZcXe^KAHvuv9?;Sx zBq^-SVRsG>dsYP>_WG)PTios_y~RrcmrABIt$2;@h21m47+V!HkuCMWGmcUk)WdE{vtF-`4DV7 z(^E$w8KCdl} z=~5}BuRp>q`(LD|Moz@h6 z_)jrLoKV&aM?1gM9{gt)Wy70f4x*)#Nub@O!LXdm(x-KVwc(3&@D&C_mit5Y?g!|3}tJJfwg*I>?!m2M^w)FAqXRocCosCsM^UININswY} zQDt3ql&e6q4+FCof)ZEFz_2&Tbs zAc4Gcm`_K)-;ARbD~+7PYG75EhiEzs{RIm+m~$#K@B9>Z%>bdI>d~hl1wJJz&wgEh zZ^MSFf-_&RL@M*?qdIn1+!gz3D8*_a_J$-y3f~!*{U%y8AN(;7F5b95!|#9L#c*3^ zhIr$^9%S8KC=@4QyB2PWj!$nOx8Ffk56>JpgRGJ&_~K6H2;;RLiOxK4IH?9kGLnDx z@{8i))A~7Syl=?$HSKMOF<@!f@eNUIsg0Goho1)qEIMKu2kdr=#}xHGDsBmK@lD%v zt+Fl@`3(6R`yIMY@B9q#%cy7{&J=Z2$y|jETPmG34Wxb?^Xq;K8ot*zRZejvO&dYBUW_?Ge=?L@KO_s~v_v{3 zF)c>HLn2PP^lxtdSIx7!2+YO&f`9ms(xv0K*}{_oIKzFghbNot2*Osc`oycRrYDe_ z*j!#eHPpfnSR4@M+u}BIp9`t95aMnJpciD8QG$4ub#c0`w9;>&!dH>;!ly!M^{SaI zqyQxkL&nC%V{)(s+aGBa>jGd`RK0GL8F5Zf`UD>PdqMN2AEi6=f~bHS+n8lI5rNgb z-cn<^$o%@}zwjWeY9p{RSSpguu;ybc<5@IOb1}N)r`OizQ^g?CdddCO7X5|4_VT2t zEOAD|trH1Se=>2zkNXWdvORlJ$IWuAdTdlc(XBf{(5Zb{09w^CMh-o4_V^0YW$|HE zFmWdtKG&36(VdP0X)jlW2(Y8+Y~(ot`j?qoBt7A1=S?r^qS|~F?va+#Q9N!Q>35)I zHRTr|62rVGaYzyJ*_fN8`zZnBwCG(-w| z^R|eM66HEKW4L_s5$FzB9@R{dms4xWENW_bmLtT!7cr+O4ukp9+`p)FM7cVzWTb!- zPGPB@r}ROFLwqFb`_|HnJ7f7TTf643(+!;^qmY|>2_uE>iG|v@WWV)_`1KpmOJai} zs#B$gK7$vnI5H{1Bj;V=(|u24624UIRoFt-<(fC*<3tu^H0M6yaBROw%gfJxsdQ&u zG^kdo9;TIkIT)9$rbs5p4qrZb!q>;C{UJ|YCJ3aVf;pxclOe(JbRb3Z?ATPW>dckC z+r^}NB2|)rV#nE?9*@SqeU}2y4KllK7u{9WaO*;I65QfQ{nFSF8MX8FtQ8hg$mmxd zAWhddnWdo`{ARM%!u&v0%oca^df0WkHnacF<5MTF2l=Jv18?vxT{t*plEaeK^dIM5 z^>7H&Z4%tj4;F;TIz~c7g#$>as)7Z%&76hoS7OSSV`;20%EK_XNf=!uHQ-zz*%B?( z;BFGkVLlTTgroi*&1{<9K-WOM_9i>-4Q8Nt&S~2TFV77AUGbA%iB0l^Am1Oj$-#O} zGJ>`N`h1Rk1HlJeVv3d_? z>#5NRTIpivpk|~>Pcpn#<&wcLs5DRjQb@rsr+d>$R$qsEcJA)6&ZNTw&gzcQv*}5x zQ}OdB$1LQxR9>=Ce5Znh`$2#1IwF^WRdo4t zSM4~YzXzyAzBdvYYxm1+>cbflXPZ0ZCas}>R)&>d4{)8<=O0-_L@yV1xAvo353L~0#)Z@jrf9___B2jeQAmOGgrz~Uotz=f9 zfYP1-LF$}WdH+BMBT3Y8Xd|!Ry7)+%@4Yo33XN~7BNq7sb6>>iTWKtI>)+6dm1Ld# zxW?_De@gN%em!cNtVVK`A=pe(B(X{MG`={l0&wEu#!Lp)Vel7wBi{IdtkFF;lc4pT znvSgjW82%8d)Z1nHCW%5Ph>^ik{J?;9hyj^S*UO7;#+Ls21Gf50y$mrQu;ed^RKu5 zFe!l$0k)l_9Gz7IE%`cM^!naoBU{qpN^ZOa$hI6Cz~VZ1phYNY;-mOTrDfgHn`n^$ zutJ*ZuN7E{96B#Wv5tsmQRztrpJ+D)GVE{P-ePgQj3KRXMQ z7`iYm;5M`5dlOAm_-m(W`a3?XHzO&YeBoI=?w9GXFHw;+jPmq3l}vRum!J_@dp$S! z7$dTf(WvU6LR45;*W^6*qX4-QZs+3y0~tAF9{-H972v&1@ol7ZFxq{yKiHo*%r9zrUGxey{OFU zQx6gfZV3_>GG(}&S#b|cbigS2s0&4}69^JVg9AZdt`-_9*W)Db!W|5$+NkdYZCQd` z7`KH{>NLzCVvAGz7Zx&t7v&7KeHC8V!9-PIIS|7lgrJGalkmP0QiY?v zmd!Ld76yeQ{yC;ROsI3M1YwnCE)`kq|_SKiG z;$$t=p48r-lYhU5m$;~dlHb^-2<1_Vz=;8W-?Z?>A!!^Z==e4A2h<-9GW+|l9)9wi zWuKYkgi~H32^T?|tMKB#$B+=#ZSk`-Qg*J?J~1bAeL-C}Gu2{NEqEi}b()qFAs7}I869`t2%LA#sJM8}$5_+u|q??Lz3TE4p z|G?vBD(*Seclz-bia)+T^{R2dIkHnT647lCj)u}*j{m;vT4+gWqvIcrzPTJl*#+%q z0Khm|Z>`%84*aNrX;1Ez85G$<69gWjUREpx!&@(UAG=kjlKBY;zXOSn!=O*2fW~!3 zDI;1IMSW(2Hp^2p&EV*n)z7Ud(w-mjBVW1IUSU<)sUIvW&8oVZP}}9lQI+szBJPF| zrtp@p!mjPfdZ#}JhVt3fu@_!BAa0z-U{cD1Lmh6^xzw)@S6oqR!s`|+JENFhd}^sb!a z^WD?P!h$5%naz%|IR>3eJzcMtqOD1N!kk}z<|3@v`~4-je7=BT8B-c2g>t+E1c+^^ z>^J7MhfSA_OnEBVjAhG;SR?4U$~}=FU$jm43C8B^mj-gl$!qNyzV% zGZe=XFuYqG)?}8?HofZZ+a1f#X=Lvumna&$$oadnFOhM7mZ#%;?VLoE?OZ+d$%Lg0 zB66c@-024q$>_!OTbpUpe{QIbj zo>}lKkJ>)DXOies1wxTpfXFYfn$TI0SxFX6Aun#rd0A-BOfp+m5G$ZZh1Mk~^{iFo z==21ZB(1ILJ>oqqWQ-UWxK+JQcb1N+7Tq(rz(pRO{G1Rv`_?Q9Mu7=u?c8sPBVSKS z8o@ta0I4;iSFX||5jTFB{-2i3%ZJ+%iz6!IIG{hC?$oakaIs{zy56q@g}qX<__WsS zJ5yMGs{ecQq&hBuVMs=#PqK+3y|^7ySL0>^=ryRC=(_obq0HEwHprQm!o!{EVlglF zk>9nRGx}l-X}3al>c8|H=oeQ=fn=smYCw!Ap#=uz z;qL&y0<%A5m&HXdyWpSNBJ1UPs(t^h3%DbjSGX(KQHe7a5#Dy;(f#y{7ME9IhOJ3O ziaC^F)ibQD^!aE&6r8I4zN%CSfp2@;^`EfNxOb!>U$-4LU0R4T=?@NA{O&TOs#b}J zWr7snNk8AHV^Ho<{=6X;@C^`t*_LB82w8a~ZfiPX04V=(&o&ChX3ACm7p{tnF!G=> zXJ^o}kE5j-&HA|)@o``;;Hi=u7-&fzH<239;Gf;kW*rG46>x6*$)c$te^<#~j#*x!;#RcXra@ zDP=)j{*O?QRAai(Ba-><%E2?0=dB!IT^|tV{$6?UX(LWuls*p;~AfgZPcpmZK=F6i? zm4(m9%iSJ}FZf!d)81QV^WL}LIB!27-9+en;`0+j7uP~b&v%WT305!X zyuRFm6~tCCcBE)m42ZEe>GMdo2kv;pO-C?@Zmnc}s7P*v29+OUy-r!qW3XMwk=)q6 zXj_0g) z|2CC2kuzC*OBs{(JO8i^M0#nWy^x6kwP z@Qi@*QF8fySLMOqiXiK^4B*DbV!3VRZ7bDAM;I&q@WN2lYk|gSS3Zo-fT*JcBH?Nx zJv8yv4Yu@n{0{(GK&HRY9O{@a-=Ioeob<6W=3ij^p8n*K_`JJt$ElQq_km*zp4ucR zBR9UFj3-QD18*$+%le*uhK99#s&TS2W7uOh*bv6>Hxl7B2%x_c{qYOGTC3!`AB!P^ zD*i>#D~p8OfGQA3r z!0cww9HMPN|1CSK4_ig1E&5flcC4hTNT89wXHlUfAp&5x#=+vJ|=!$HJ+dMs?;nR$X{9Aa^0K z@YnfCbZm|_1)jeHNDTY_P7n9VYk=MzNcZc@d;IA_MJo0^1rGK5{iFvrR|7ku>FbTU zm6h*|1lcy$`OQWP1G57#!|3m#c-UJUoP|hI?(eP-KKecu@M0p-4!PPoahcyODJAg( z2PaaZ3vz?&Fy*|BZrya8LctyU`(}HDM%~MXKJj8bXh< zVAjK!YuEV}UQ?=^sBY2?jX<%vzkS4e`;$-z-)5|sgr5LU1+HI!C_}-@GE5>tj~g0U zATVV|cad`RNR(Uc!RU+sD%21Jnj7NOIh}4tUaFVd^^cg4g3cps1~Tq)LL?5Y;X~e6 z%0P6UST+B&>zj*I?0CRZ@Bq=ox-*#v&5tu1Zm$GP^9;8g64yvRPwNHiTYA6K1KuPY z@a3iX%xPSj2kUBhY`V4BBsS7qIsSUL2lCgPwP>TG@D3q(?E zfcUUCO1NWix%HLn_ypk_1K8G^)K+31!u<2#dVun0S0Zy~K5g(G^;kRxaxI;6;g~?LIyV}`xi$ph%XLmt1bQb;14C&k6|&7fqC%g(?|4m z;i}6JJ{Isc0Z!Rr;_;l;hjDG~2igk~RAW&WY_I$G>pBHP22Y^7e-x@D1O{NgzdO)( zv>NO$ALx(Hz5q0PB*rV$V#_Hhs?oeokYq~<{(Ap9n%{Bb)l5)%0;n#-`&vWs{4Cf^ zvI&D7K5s|+G?QEv{O{RL1>`&et`N2L>)x!Q&do0FPYj`n+e-o#o)R1+{KE}L%j&-j zk`*|B^hBlTPjkur+we;a;A94$=5nB?uqNMt`HfAdygaj1o@L#UAFIICd@g9_oB%J1 zZ}o?r8Jc3!JZ*5eH9el^>LRPDa9XjLB;-yFrr(LhCUyKfpQvYzSNUo7g)6+Z=U0y& zwUP@mHi?^XMSD719Kt6sA%1`y3Z>(5c85*&jZ@E|9XY4R0Un-dJpR5$Q}{+3#&n}) zm)&T6S;$e8)#kdIO6}EGe6sBjBmSsVR^G=R1XsPeSdb}wnLoy7(7*OLuRL)H5Lgd_ zYmBNXQFoO*anaq#S|MFSdA{04->dxTDXMcXNqXYc_ zM|eKx72zDWNTAi30)d`Yzu-$7OkYIk>(!jp9!+%FkeTI{9&$MboA`soPYbwS1iI?yvguKn$>-skb%H2X_VM|IH{5lf` zOQc3EwysCa!_uT8W+{3;FTYklCk6@1omt%2Rk=f&Mf!xC*R2%MU&HPfyuy1xXTC>7 zeUyZ{Vk=T6nu)Cq2+uW2#4-*7zNa>pqH8(yegVgb840(qa@g#rMlVM@EpRBn%2xfp1I|pyh zUG=YbFOX~}yVkwFPXoNqkNQvmhUqOwdwhodO-!IO7Nxnu8UK8hQ4Ste39k_K_L5Yn zwl6k?-&c$5o%CsV^&gZiV^m)7SNwD%1P?f?Y9`+^AZ6Ue$sWa-@vv=mQh;tmu*w&? zsU&N|p432Le@!a4+#^>kjM>q%Csugp6_qybAt){tF57rXzpa&is~-nEb028Te`yi8 zNM-#W+pi>dA?&eBa8F|+CgAS$a7L=p_19_U2J~6?g}nd#2FL=`9xY<&wHKEUE7Fg1 zo8lx=FNuB%Rw>Rmg8&U#QMGiHcR4o1384Gjh}@(REyX@(yo&CCR+;<2`6 zGkZ+S4LM*krbJSis%_mBUWjZn3Ar1Z-4lC-$aU1;m)<=2xv}=;T4j}Des?gHU`&Dc zc)j8Cz-WH2V0nD!*XE0RMfew4o&1FYSB-pa*!#t|BgEyiZ714;_UBC=E6AMUlSCjS zy!Fndwc$o#^x!9_(|NMT0`G6@=%5L=Ui{_OtJU}m!owEj?41(x71~(2@D&y7Nlk7Aq$SfSIJWil&yvCX zH7?>qEyTeYIP!Fq z|B=-B^MQF$W{WS+j5=R<{}cf2B|SI>^vR>Tfk3UVDUIB-e_uGpWI{mvEQ~TtE-%x? zgvLKMs1a1;LEd-1=++9u=7|`EW>7JkQ^~z&fD_Dt zos0zC3c}sd+jf2hraSBE`oS+d-9|V%yj%TieVfI^XfZ~jAS{L`x&~yeCXtA5S{Qri`#>#*U|fh4u!+Yrrcd+izY7SHoH;uZNN_+NEOm5#ZP#p8{by3&~dGi z3~cUyvHZ2Y42j#D$6vxDmzM_c-$D2avAUp~dl39Gb$Jbq+1Z)ys=aXO3VqA>;9Fr- zCMD~A$MbwYF`%&|Z%2QM_la^Uupx`uvO}4B&q2Mv$lsNQni7JIGlLG>Bi~(@4kyyN zYM_R7`U2ve(O%kJ>De2YN;gR^A5fZt+XWj_f?9C7D6-)gGEiJs7k-va%Gsh?Jb z(kA4HyS#<&6VHE#&nx5S+-l;b__b9kCmty=ECG?2aO32UyuY$zZ%nIMycCxx1&Kn2 zUKdf&4gv{%DS=n7z4QHoA*NI)J?L|=vf;Z(CR zwhCrNtyi?$hk70h!Vs|HV4njSg(n1dY#2zs;FnXUUHKc)nH=g_R7#Zq78N5=c!7SJzrf8)D< zRM4)VrrPoE_qUS`3^=l%=>5GkocvH`_TU}554@0d(!WzJ4~WD5-C#rfesxssMXynv z8ygNF3=hYF0Df-UOUG0*_8X{p@NIeWJfF4&v&$E)F;h!Ja%U<;zI*z9VsYV(V?kV< zzYTt~rCJACeK9re>#a2hMQ3_hN0ZWJV~FYn|MfJO6f><$acd*X3HHJOH*jm4EAHb_ z-#nmFK-*S{5qR`h3-9^^iSPu&}0aMLyf2gB9q|f-VOP7*gq<`v9!%R8paFHV(eZUDy z#z<1*kx6*bT|m2)Kn;}%^Yi{9EJ`AQX-1zLND>kwTYag0$t4m{J1>;f(9_u0?0`@u*5 z6eKEe#9yNPf+b&;kTAIGPn;72*`#~Xzv~LFFBeu^)^4r3xld_4<4V_MR|~&j3xel% z@M}x>@$9r+>dJd3D^zjhY4rMN8C8)tCf7Ss+!N%%SL{Y^zbI#*Sw6qXW9Apgws}h& z{PcyH(mp)_$gU6PYbcBjHU{6Ghm`uDvR&(|AqCYF^Bbdja9rh;Ui=B^Vo; zn9gfj&l8`R0d5oe`wGy@cc3s2<{x$qtvABARO7K=Mw;CNg6P(}z@yx|#>IN*{@wzn zjV-pL4tTOq4df)|m=ifn(%i&;v(rn)G#=TPX|~~j75r%mZzk7RrJ+Oq5WlYR@c!S( zsC20W=@YODmWmVAY>=(NHFx6{{E~R$$AidV$j34jwLRtzDR}y#K@Hgn<_d-J*B4Kl zSJ>+|2efeSU4V zzr2CRC2cSjA{>t*)H02HE4o{pMoc5%d&YJG1C8GxTLbid2D@Vf*?7^qO0d7Zbya4_loj^0hwLD%^k7{rLAyJf8nU8i%8=J3CneOnK_&3|VbtETroqC}P*;xMT=GARikH)dtNsUcPR@Wvo7f8YsWuj(l(=AX>Vkuphi)zqoVKSk- zb?T35%hurlhcs!=SAyNIKH8<8YI-|#6oL{yGB1+4+;KVeGbya-eAP?-=aDLC^iHU>v2lObZYlA2u`IQS>)(EdA@9(F`oxnX|W4ttmI}vbJVz_45Wk`#tRdF2~P* z#cpu=W+{Pu2=9u%9F!#c3wx6PfhBobl`LIvfFZO=uojDP-v3P5Mf@oFjZ+ygX#P+x zMvPt{NXKH6L!n9FD!?_3SH8CuZp7b1A2RmWtk|xl6tu+?=yc0@5lMSHLrrCja?^fD zzu;W_zyiZ!+vP?sdr=wAN29_}VHfU?+-(kRjtd>~Mtos$K^vEv59#g|IhjtpJ>_C+<8o7wbeZ%O@lN3ob1k6*%L!JW&`v=jq9+*UBPAc<;*w zo)agpde5G`tYH2;%^h0EWb2M^8qaz@HVowvRYbp*#N}(GRn8yWioS78?MHqiq`#7A z{>9QzlPq_IGxLw-F?yj%X0twUmixZOTa)GAXqeRIE^uGGPj=M5zbg@tD>(EfK=Zm& z)^@w04@jIsddo^(Fx+GIGA=!2vEv5|5^~g@9KHE5t3b3ova;dLTpXXb(?!shrB3g* zWKLVKrPec4uCei#ihQ61kswms5Uq5bN)=M}m!^ zSJ2*O-BY+z!Gm&0+4p@okl~8jkwV*T)A%-PtRTK1Z%IqO3l6@%1Nyh;=L0$HN-m}F z*;_y}ztH$4QW;st`PE`K#oq2STO^FK<8Mcw_W7DB=EeOE*bmylXINg_HDq&S%bCE5 zZGg?Zud21?Cs7euM4e-_6G$e8on5NX*N4Zs785p<7*E9bVqyZ!i-Z!vCVjbsKWl)T zH#g&4bjf^cKtX(~M}oJYiAR=r=>!_(^UdOLbF4Jkh^a9sRjIv zZ56i`{JHD#!(Sk!$w*NI7evNszKLG$ADo<)qeEr@44H=xppuBPr^JaszK4o_2Ex%R z{yr{rIaTSM+2GX4D(jyrA&k=U3xTr`q(I4Hw|h0pIXDdQzH7|1E$!U>Sbo2WJk$#V?z0Apn?3i` z!tYx*7r^~c;4O)iU*_A&yWfFi3x`>1#rS5kX!w3hPjqq^UUk4`UHAz1k#vR%JaFa^pO@n&3LIGpuGhlsD`Ma(yIVey5*?N;i` zq74M?(pc}mr2NI2=o^ZxTk!@YBQy2xK9vtQM5lzq~MDBHdA{O}I ze&KDF`Z)^Uu!)fhy9r8@H9l8oD`W<%#)G{&CjeU=2>O=9jqG`-{Xowc2g zR*5p`4ZaI;1Fs2OnQrr#HZfz_Yo1dX-OR4cd?oyHV_~1>oUipS1iME-sw(bhJ_Q?s2_I*L5lphLUB2A|H8 z-6>j9Yj>jGt6I_!#Q?4NM(6a8Oo5sGF7gp)IZPNWbj@X~{n14k8F_Llnn4?f8M%RU zAqOrT;q;lfJC1a?HCjT3Y`83!3Pa0)(tjU~5}xvTeJY0Rk~p~&rfW_fn=qc7?;TC& z>)+Na*I3k0$}|2;E`|X=wluVq4c|(hLdh`Y7HRT~^me{wF}5qTA!3;kLE|3x*T;6LIhpmNC*^K7;6kqEC=U4tZnf-*S}h!H~<_r#*ujxVT#d zzIg92td0C;{t_Dzs$ZL%AiAx!lT@*Iz{j3PLZn}wP_KkCy*6IggmL%btw$d-A()1{ z{R$|bpcPEc+ABE0r8To!K`$k5NaaJBPEWo2*1wm6m7@Sc?lYq;6uQIs^GOxRO5UX| z-G<|%SL1b&bAm+yt*>~J?=3f-gzCf`zVu2vm1ISOqL7PCeD6Zuwq(##7+-0u@B5H4 zK=`c&*!sB`ZsPfpEK~2Oy`4@FxKR0yG}A*ML5N2hJUx zJ2qQ6bV#Th(WFa&@4m5{IVzlAqFv^~oLx=f|In^*nQ&>zksfD0H`q#;7!*) zAkI@Z0q(R-cKA&IoJ3Ow41NPYUfUvJ-U4_Qz<30j$r}ph*`!wJw%No zhNOXMOcQ8Y7j*~L2k;R-7seEFVbQv|G3i?d=w0{1OmDTAKT;HDo$%s1Tls-q|23f1 zW}z!0^Cz7S>u{j2bf4xGO3Fq$Xg;x&BQPxC7=K&a9I_9qFE@N^dlG2(S5j~MQIdy9 zp?Y@%mt;>552;*OOq)FpzQ(3Oz%M8WU$Wbe0&U*G7s~1`wKP0J@ zQ=mofyW?eI7FvwDZwv@QQu0ClYjF+HYo~+n10I$4+l3o+!>}OKY4wc2muUzgCu->T z2(t^4CBx~Rod}S%sZu{{yX*Z;yZKJjvAM^WXQCKY$ZDQ?i>xnusZuimdHVv`-D_-I zVL*)=P~Vuh)?}>=3h~LU-;Y`{Jc+=WtrxVr{z0md$Mx2$BT8l9T5IcEjo+0HN&PZk zI!k=jRD~ZlVdSv?rh{R7J@W7IhKKp#Hn?1$(E_xQ_9^%as6>{(ob0&@V56y^lj>i~ ztr|Od9}m}gae#B9j>el97BFh-d7tRh$F2byQ>Od>Kf8z@qI14US@CS@V*D`QxJ6Vd zxbkKf$3L!I$xM;w;rNw%6-pVq1F-fQB4}piD%Kr6)TX0SsS*8TgFs=dGXQ&IsErqS z6LyfRk_$}Tq-%ntMv;m6t-5z=aCjr}H0vH^-vBR7vzwrhHTQ7ZxTPQR(TKU#nnDU{ zT*jspe9?k!d(7t-dAxya@IT+=BbugvzCye_r>Fh~)C(E=rAkYn6M7YpL3Hc(WJF_@ zVGrjwu-yEubQ8)q`2jsZYVsdvdRo_X`rb131f6uf$o!yJbV@0H7o^mwcl~RA;w{8i zEO@=Y_Fj2HP!u|mre+~9T|d@c9P~TGXh(EtnlPRSS{T+m;8bAf7F>7Xu z30YwaRYvg`gjOh)9NUh#6+#IIHR?31B15F4aiHvY1qH`86UK^5AM?>!hXoC`!d#Ne zMhE0T!IOKiO!mh3Ll9!yI|sRi;GXuiZ`058C+~u139Ve}A(1;A6OEVJJ-Ym*z6|!i zcf>96O&D&o(tF}?c0i*`{<4{jB@F1#H=EN!9VjJE86V^7E#3z%wKWL#z_NQ2gtc)I z1^%6_>}Bsz4M) zf2nfSl&nZ>f-pofNDxW>{p0nZ2lkTIs(a7bJ4M_3*ls!l3gF{(dsqAf8d(QX)tyx4 ze-#m#XSpDEcVDX8BKfsq`GilQy7YcvM_tqM0LOpxW>310d9@*?s z7iTzCEnAL6)-#mJDwPdV;_r&>Cam9`D~Se%w*lcT!iY}!V7J1vQR5-PHebbmJ_U7p zSA4-v%p2sB+$$a72w28{jrF3gLm|JHsz_8zY8psL8HAiteXrPPwVVWx!Dx6#$op($ zjI{I|`P^&1#bmsEB8gd_uOXc`6ZV(_<=mc3dpDoY=J1hp;+vQ0$@U|_R?epEQvr|t zEbpzY9OHL~u~E|iXd#|v)61PUpKiXw5^W&OPcor_HwHsz&%mFMZmbjttZvM1LD&=+ z98>va^vMc5X5#+x8_SZ=9C zxHs0GJW2fpjzvL92}%bK=o@SfL`x4b97+3aXGH<#A}uupf9GJJ)^K!Yvnj77U<+fg zZ!FAx#VGui0J|F{@haA5<$3@sFi}! z*<-brA$Q(^45-4bDLH}JJiSF3Vh}GV_zKzOL7aRt7(%{m;1|64-u(Su?gL(S5W8GLO1Kh1GSM_(tytS;;+npL-?G2!QUi57jeAjt{L7B`16T-rliVv|`#D)c zJzSZDzSep8af%}wBR8F7X?u~y|BAKv2-8anPC=sRkWJE-q9|D%=7 za|L9fgb<5BYMBF45;#~j#rnE3@3?8U+JjVp2yyz_e?*e`NU6;}_VHkbuvWXv`_o#F zsH3TcBETxYnQpXdz6UIqY)F zI-Y$?=F0uczV{K5+(rh8(e$+$ANs`j@IL?mq3^WIi_H_v=m}|T-z>i0owbV3jg>2R z-l4}=T$}ke9apLL{1K2;IC{^A&!Ax8=gt)ehNdSf3BIA=j6}}0^o%!HocgjIAiqK8 z5%NH*mLubM zUKv1d%r;vA^W}@_tfpcno;{h++5JfxXvP=llbzw5a9LH<{RaLfs6g^z?*&kff6t6> zA5v zs=|5EV0wtDNNKwx^6Un2_g;pTL-6hrMGskJFu%nCCyTVHX!6!x{Y2FQ!}JjR$75uPEkI;f2fM6ZEcyJ2EAo{$U;ps+W>$8* zC7a`HAvXZi{6bS~lfU8gY+@jBUbCGbP2g$ywAuM^cED5B!{B{NNqdhWy1s;XhfCl)mSrP%;pV3c$Y`2^$jm^#@e zq6Shkex_-L$c8KpK?gj-(C|a&EOsSDZX>>I87Co|uf8-eLgM(N*W$T?_e>A#%BIqG zeU|$D32E|KU0`a63HI)xd*a($EfArIBp(>Z=B{3V;jBEmXbEy!LqA&BcmY;`qK@lR z_Of0nD>$(#*tK}Ky0-#rQDO(L zi4ymm0v6p$c6~Hr<>H0IzD!c(1FHS+LMpV<+s1eT^=<$r3<-2PuyAz2G75w^B)u77 zxiTjlBNyues<3TFOfCPpT})Gi*Xy5r91>Z+M=-sz;Z>oQy*JH%v>|+qe_oia1PsY* zMz8HDy%KK9{>|F9F-{UAtH$35hkg#`uUm+{37S!M-5J5NFLedH@UmoW?o)^tq^}9u z7Sb}RHM>Zcijl3K1f%$raN4=0-?D6|x#?f~l z>NY9a+a_i6D-;a0fsto+?L{mqo+$SY1?zrOlU z$%0lw2@;EBJ1b>^r{6z@=1M`g{MU|h$`SRhO}{exSJc~asK}tnfHr;&)GHgiYBzik z%04t$*0mM{wTi+;{EVnp13R^1p*+TFrUd-`3T}d9#f|`J&WNGR3v8m2zSc#8YJAsM z7-{R#UX6G`IKAYDam#z&N%F=7Bzr0B2tW&)^CI5v)!oQTC2p!#C#>g3tQbyMV@`oB z)Utkd&9g!K@!t5HbDB^}!KUFg1rA>f18A}mA~sx-Z-b5h|pT(T83 zg`0^!2}R2XtH9NR$j#L!pLxAC2N2~laJkX0mSMLChBH98Uct_RDyy|9&=^gW&9^E! zX1e+z9^|>T2xYv)cbUO0m~w(J^L!eD{(5~UMsuPJ20A$)bchh(@iA!lwD>%R%LPtG z)`56zugU)NORW?ywB`b~W&8q#4l#@wPE9nyeL zuZ|UKJf(u75%-(xQYYfK;5_|5$T|3owAQ(EO1Fl~+7#Zj=b!e5PlOu?BiS6yASDbh zoQcxLCX}XkcK8_?xG*pOZJxWVBmcSuiH41HMK(3n_0qns=R{m((vGHWt~t++f-nK9 zUR4R7Y~T#`$AWzki3@o`9ukA3SG1qFdO{FX^$*m(^3^MM=g;>OzKG;R1Axh=DX3%L z);pG-p^{vrW=1wJ*BJaQH{p^@68_s7k^Tw;6xUO-3WIS z$;K$r*m_`>7V}>gi%o_jtLPV!zxNyBfCkag=L@TqH6&p@(ZvWO6)K7?zz3U{1C#@n zIH8DS=|5R8N&DNerG5za)TCOHi^8D&D85gkVH|o7JY0yszAC_UM0@sU_I1tJ@zlh^;5WLQ z=wbaKKl8A^;r9y&_BF|8rZ)bV2UCR81<{GGYUq~M!dYl#jy9!t6`Tm%b2Sw!IMU)i7ZxV=2P z`P&Rd;rBB+5QJ4o3g~5mgYIQH+qxG4R$2uF#ib2%NcA zlKI6$aWw%->;i?eUjW-gkFJ3Q@hXaD7O1((go-osvNPd8#RBvVHk(GzRRDKCOeXB> z58LC?DIA0+kFoiC{h^LTayPR(zi}7e_%A_{PVc3%F~zECxm3)*QRgQ?eoV39qg)lq zH!#*+2*D6?+z&5`aMOqM@5n5#a}WPPIv}z%6+?66X++;;As(r6zz$_pMCaE1Y-GRZ zCVM-n0k5JG;f2)3U4QR9ZUMnW>e31_`$HRk_hkKFAQVIS4 z=!M`q@)R_ZL_Pb(=!mnSzi8$6;<_-#NB6)KFSj4NMu~W$g7+lmuMOaUtXM}$yaEqj z5H6rq%w(JW{Jq{_apjO%(7N^-V9SYZgLm6q(SAc{j|?7{!aG@ZRlJ&2@>J-;w1*6Q z6o|?f7vj04-zpwj2XNuM^I!F-5rX((fSb||{7c8=IrAP3nP4`7uz#-T(*0swY($&lA|4mGp)U*Y6^YN)pxsYj(p2?S=WKSS)E?MI}Z^b z(@tT^W4ltGAMT=3raN1GaoWSq?w5!2*nJ%M?}$A1=V!iB$HA?Ofbm%fUyEd~-*W6H zci`R(bUamo<*YxDFodk(^YB>PY4CU?_^PAfx!+9%SN)`e#I6vj0coDZso*mLWDm8q z68@LMfb3&@xwo*sjOSyQN#FPSO(2-U%`|S`C(aANv191>@aNAy=LiDfGjW%Kpu8e? zakmKmrSz;m?nb~vKXYxSQ^HDm6-Xkw$33{M3!%S+ej^R&Mq#I9zx??AAwU)tJ@)T2 z^y(x!#lRsnb+GKG7W#AcIY+T$n=na@f=4Apb}24H4fdP;jg$_oP?{%xGNv-1(9<&= z4)^p0z#I90^8zFRkys=B@u2?e633_9I^hM#>rQU{{@qqr5E+meFKvXtd*TxRUTZFw z^SdrzkC6&+*XfaqLMZUljYb1`uKun`ebAt-X;B;bd$6rO0ATPbqj60i@LK7k6>a!c zT|iIWG$)Z%7WRk=bg?bnl(K`vvo(kGd4JsJ*~m@a!TACVk5Ww@eVQve{d*FE-ylLS z6o-OJx~zO;s9%IHepJ5OcC2&Mc-C^#7u?{*DZPs4c4z5+A6)!uj&kBhgi#GvrRDxy3i;b%Pz67b^NA*Wph3)Mu-qWM@p8RP>w5tFokoOt zc%r&-&Je!7G)8vX4mt2q43JQ$8IaFjew&mJjkA-dqMlwmytnryoLDvB72es8Cf);- z^n)SCkhXWG7T(`^L+`F8-xmHCWY+XjHD*^-Qo45wpZq)CD1GTYF{?OpCCqdo(IPS&D!ZPin@g?K}Fr7hagzxO>wn2){dA%O_t5`)E z>ZDY!TJCMrqgAmO{o(!~A;D6?Kk}>1rQ}MF7tem|!1HxF+Ft|lpZ+Xj9Fvc60|5V4 z)l2?*-aTT_s?RG|%h`S*Hd=>U&V&UFpYYjvic==aMmuhp!c~&mj_>({^BUrVfcI_Z z$CasYAhv%ksE}RF3LftsW?!yfvk_j-8aU@eDDD*G!f~mEg_84O$`P_FaFQHd{CoXA zI3_F9E~!g73&!t~rv$xo>BN8EJ-Bcuf>3oyQ2hxn|r3}_7LxTS#{E!VfJjsm%%FUV zhDm}8k5z`tbMYh?UhCo)i~zPuqSXb)_hP3e4G=rEFSLHzMPlG7Z82oj(dXb=F3+`Z zXANV)9m>~tP}&I0kW2*D}l z_A0gcqkkhaJ_wEu`iS8A)gB|6UI@C76eopj+#C@>3B>Cc?2KH*gb=-iR6DB%TrjX- zc3qz_re_OVCigOcyHoE>zxJ$cRGAy>`av=k;#4{vA3KW~A3(oVw-a|GvCL4!uC@+2xA%r@4>Ff+eec z2D35Nz*ex{Yq{<`VcLq|f*sakFv00yp|P9_@~sZDvCtmRi^Y7-0O8wCG%|8wVoe{mC29cj@$ryisWy-#gLgW+u(d*vI1& zJ>aId1v#Nps_{sh0<|95f4bH|1YiZ)_#&0~UX~YB0{gazJAgG=Y)kEl5CN8CYADLX)K-yd`^oHvrpZz6DlO7d`K~lR$n~O--Udgf6poD zwG>d^2`X&dp8E09X?VL1OF#96`#;b@fZ7k~cR~#+`(+EbKSLP?PZbEKA|2gsUX}jl zDj-E3^0AO^1wZ!3)${V|=4$^%RFB_Z*edXLq_q>1oE<6dUT-JcYN8Im z<&WkLz8QD&Xq3gPVNZZ3-XUYVPeEE4T{HgfG=zv{!CxeUIm;h6de4s+8g760`O9f_ z;{w6LcN@-~5daQ#`z=3@`_&z_p8xVwytpSCeglfIuT+EW6bA*igxMnXaQ7g{*SWF27x=5s^4PL8;rnC0AdAV?Dc@_qP$5m4OzL@*4@Rhh8(vHI zb!OPRKfX$HRy+IB)$mQ1PLJ2&!- z2))rG8(vWB0lY7yyzI4D(B*HhRS2@439_9Jv2959q&TjKe0{GxyZH|+Tg{6|=nod@ zg%1_`to@bJb{RvQ6YY=qt8E^qz(Yur8H7H#J?JSQ)RFxUnFs(#P?Kc!Khi018PuGcP%@Ei~z9 z$qJ=R#4$0V?KGdnll&Q_MSp_(+oDapr(J`$gs55`<4(=V0P54QK=g(22t+!$&y;qr$Eb*X%@1^L9J`8voTKEDQ{Hq z72TA5lNX-!Qs-!@6Ra4VR#Lp&iOi^Z+IX3;rQoAR*FvPk!KlLvb{029+hoc$eB_5v z(M@S7rm9@cEIAlMP};)sLKNma)4k&q9neCe^3{+%CkeKvz)Mysg{4?(UQ#OMTtvx9 zoT)kKdS-W7v8kJ7;!BLzR63Zs=$GD6H4dw+)4%`=NrG%ui5ZukFFMBaD-T<#IJKoG zS&S=ScBIR)TN~Myx*m5F$AciaH1ECe>P~c`DAXP{m87&NtgLN|weVG`hbt935OmVX zHSXu4sKgwY7IZq51eD9B4tpt;Ob=qXv|X?y7B*U!RyMRLV%J$1LTh=hZ^PG&DX5Yn zcS%T$oSZHRF<&#R>tXL_thc%zg~lj*o`cM^EaFVta+Z(nG>l>*QrNa z6*kS0HJ?{k7kZqvM)jI(0T4<5v1@yxb4wRXAy`TlRTNshw| z%c%=f*m*RqEw4H?ON^|BB8#QvI2EjsYSf(+NNJTV3EgaaPT5&8#hBB+wke;5+A5l) zb0+tztAtAwrVV;EGeaRsl{S0MqqUHX4A;ikTU^)DCz67PwCkcFdQGTo000wr0RTxS zL2AZc5~*Kx6!SDBbWvrrCdN>Lqe7t%Go?%_3Q?ruxoK7BcSh^8jYX*PJ0Mvltu>1aJxOLAibfY>5^Npzs}EQ` zZklpr?5C1TDgnc)S_$y^&0G^%uD$m1{Drv^7j^f8vuf%<+ z=HoSpLG%gb0KLk`rZZK9$1D)2GCio{E*~g%BkheAJW&|(OFX-qWsz(`c;6sk)KNUn z0GklTu>DmNk_L?B70y^28Y8+KT&3I&&Gk}DB0*_~h5_j|#GWr=5Jj3KZIY-sF0H`w zeHemmk2Z9KaSjP9KJK-&HmIf#3yB9`j`=MGL{56b+gu-vh+L-t-d$LFM{eRA_(5Tm zILG?$c|~p}x)vcKQkrroLJgES(C;B*%hnZ1joT;jz^+u%ZI1{Y3|k%ZKYH>bgAt5sZ@+TOx%(# zgy?pSCff<3!`x88q?XG8E?doO#Czo2hK5(RBmvtE3?kAFxDzEalIcHZkm$rFsaq%m z`y)55t!b!&Ftw7wL6rlDO2gpu6dR&jRA`oSqc!^q*-*l4zQt0AN6$zZ*h~a3O_(GW_2G`5{i;UMGT@btSP{V*seRlh_ENL6TRF#|XR%(4tI4 zNSa-1+!d3#T$;~HltJxzoH-P75TtC2mhl`@3v(F}q@XYlf`xNTL+s6>WXUrR@!lXP z7)DM?9wdiJmqW9jL%`8hSxHF=2(Z2kRjFK?P6($f){F&;v1$o%`QfY(n;$}CnsDzu z??4uzF;Tm34v6ehq?l<2b{GQiM?^;;128p6VIUAM zU?CD~zPJwn*3t?|W>H-(*W5Z-qJa=mz~wCW>_4W+He3bqBF907HB{+Dm}5c`!1es& z20|@osa9T-$SRpvWP)5Q$u=hA@Ptg6Od@T-gE9lugc_t0RsRI!jzmsqhOn25L6vL~ zW+q}@SXeWt9`405HQ1hyNJw=n0#H@HMSfz{K`w=o&NQ$9P~48_ia$@DDFD_YC5$nd z57OpH8+ZD))not`CyDN}7xo2FAnY*UMf&9%*g%{vi6S`Wh()rd z981PkZgoS(!z`E)1T6^&1S1MZnzh{I&1CI5Qar^RIbC45D&aaG#k80@;9IW>QW;wX z*bD|nm_bIROf4VTN)JIHkt^s`$Onh;%L<^+VN*>Xx3anc!5cqiu*pri=x%n6(>`i0 zSq96c;9HHtFob|Pbh4HzMb286nNG;~w5TXNocaDpYJAv>Ru0ijyZ&lL{qrc%5G*4+ zikL9K>6xOW=AU^YS#h}4LuGuST})kbXjw>>wDc*}EK6|8@gFzI0BF{b#T0Jf-Y=A! zCmTs2194?Jnpl)Tp~4eVQ!t1$_^O`=v7IcVbz(sl^uivjW}35*Z%I;;(1|(!H8N$U zgOj$8B}mo|2Uv5tFvdj)>lX=5xbUUsV=$vo$xI=rr65EM)x-k8PT)`yj;SoU;-cy| za*U5E_Z){poM2MK5H~7y>HC*2sv8O$z2wW3Q14DvA(%)%g(IJDCV z1PCn&I19mvsDjV}u@wjL^+CgbG^LZv5vt7gK?j{J01Ku~4lO7GJru2a5;5a^F-C~IxZ#TwxEGR0)* z0180Ippir`idd16;j9In_8gE6IAy7kANtc<9UXr|ug5V0j&5=1&54DXN$P^$-Db*lwO~0t}`!X9K%oWkJavAr<3REBv ze?>kqup%66&ddKb;2_!$iHs)*0fC3D9iSgp{B37QyZ~&~MlWEl;Nc2qPAuuipCGtG z;S;o%>ll8&l2V3BgV00d)WH`A*!EYd)oP-3)4Kx9vqk{`+W&}{C# zp+!|_)LIgzEgV^JYa;Lf7fJt_UZEiDJRsW@$x%}p;&^Of7s${uq`FY-JuGw1c#B}U zUglHgwH9VEXqn;_O6hfBvt*Wpx)zivE0wRUA2i+=G!*;dR#~As*vqae)lpMn*n7oG zt*=L67_qG*v9yI+cS+DY7Y5L#p_X+-u~TOkNh?I73MANhW_!j8q@t%%XHk=KufLR|Re$xymn%*ES+4siQVR)1lWyyy{xylU0wBlJ&)LMHpAH#VhdJ zw&8j= zvCDc(7B-G6=yR0QX=+qClVVo3qSq|(b|BZGQipPhBfgh2Uq(WeLZ)c0ST!XTp6YSRw5~?0uUfSdR4}PNs>2}2;nk_B zRH9`~Ty)_WY|#^-(1dE_EU-PW$E|L(u#1_Jovh}_?L2FENv1tQ9cerqeqQTvG+AEcq%GjHMKoO+1Rl+ONbK%9P5Q zW?2=IT+3>!4`nb_P3klhQq?M)jWoeCrCT=M_M=fpNv*umV?Ibu4u?IBgKFE-hL#|C zC6l#mi7EAo%?RR1V%J5~)I(83$wTk^lwgV_}tlDclo)%uB-=(eGtgfaY#Q0A0wwXQt*bzM|9wsOhWy=pUa3 zV7ThHgJ{YSrguVr&oVvCH^6DL+9POk3<04HX78H=+#>^&pdr;Rf0WF8(^-p(-NAsm z^hp4u%vk`9Ih~j%UJcu747mMyzLX<+(bow057zNu0m1dZ4814?S~@pC8+HiZ6dFKp zCqrG{1R43toW7hno`C(%;BE15`ulhzosxUD0q^60AFFV15d;LJ(@t>%aTHz)5wpBo92b!E$^qvdK2DGUG#SI0 zjSzDpA_%ci86=?am(3=aQYgNAjLtA%RsgE53mW02Rfdrw6bh=L1$4+*y=H`5~w%rkp1m9`!PKtUXa_`*yE)}FJun%0uM z0ZQT189neOU_Bt=q=uuNzg%|dy|q8;2yB|mQ6NYhFudK&r7ytNdIIcFwV;xZ`?|sY z51dJnoC5t^2xPr%x3PW716__DLC6or$}8iK0m2JB*d{krEpQXWf5sKB!3y#CEq2VE z{W&{ieM7vu*fSGOQ}}`STuG{LR3JLaGVsY&ePc*ghe4{BE{_4LTe0PkiB$)HW5_g} zCMoE2$dN26etK`kgBBF)4c;IGpAz-N)DEchia)8Xo{LFU+Z=w`lR_;e!eo5FhfpAc z&E3-;i99D$%;PTp3WUEV&1#Bld21V8i)Z#9w$(a$5K2+}q0kKd@2-Q4Z2A+LFUO-( ztb@J%GW3U_r9eGO-}JzE=5r3M=Wn)wwe(a7%lrJGbp%)wn5VEIe;7LPLE884{$1%2 zwIqZvz>zOtd;pAYLdyei4$%S=hnzFK3&P4jDmT8^s+yCc4{t~>u0)LR>A}6ZXJZNA zz`R0Ce-Vx8FgZNpng##!jF*j6pY|Q+aD1e;iteKpc+D`HX{6E z?LVhhf?vMOGa`S~;#G4YDxyjdQ2a>!`}&0+`%4imE#C9h`wgs6UnTqU9`6ofI{c9d znm4deqTR>HUL(04(O^AAqWwJ^{KOck>znVdo`9gfgMmEVFPEj6rM;U6!yKoY=a|Z^l2sWX7_7pAqL}$nz zKj(sqioLCu+_WZ77!BMAxC*Ku8a94?qe>~(a&6w~8`TKIusR0zx)N`!AFy1JjC0bz z{-`MOYy5Tr&}L;qsLqD!1)AuBI+Ce^zQo=srL`iac2$9X4(gH@@g0pn@f%KtCX=sJiv~2U z%6s7W&UoHgR@fgnc+GDAKp3sCX~)6ixJ+$VRO+ecXx%y4C|p zxGWClW73N)+#^*k31YHg$kXlkmA~Kil_$TA1N!4|(I6huD3&#a>D{r}z(B&w5zP3y z=Q(DeJU32UcUlQMFc+}BW!q?fcqKP-CnM$-J4EJU%)i{-!D_0OKehNVXWd%T6bolF zFlZ9}3U?4XSZNEmdvX{7nL98Mgvl2ooE`G`%L-;HSfdivg$zJQoM4 z3Y}qxOQ4bZN>X>mc|2e4=4U@}y%L3HFQb=3<9mE|5R(JqM$u7>JEP_MZ+0IE?sJFH zAE(>ZE~=v%u9AoFi}Ufyg741}0^5ZBu~enwfk%D^S|X`pjl}r6-0WK-CzmDc(^cu! zL zC&TAi^{*cRzX~(7RHdX27&*`MMi6=r{Yf1Ntmw_P)Tp7b6oS;D4bg9{0YaceheR}n z0(Q&~f)@S0;A^{?1Q+Ql{=P;w_#ai~YTUjsp|>c*Gn=RcCWd}_*)PH&bRz}VH0pt# zNKojlj_Wc8KqEU%fx`SqhPla6AQO|XcrIdrQn{E#oe*DKAAa^n+NzTgedCAh}6_VsJ8VbnU_YpvS zT8oF$5P91^MhITdB`bMNZdZgf0f3t@!2C`X_gO|-W%NXl^0i!SlCAm6A*3)VehDHP z9~K8G!#DXu%JqEgGPGE$lZq)0yPQX_sW~Fm_e8gbW`{sGkO1<0aGE#d4aX8Pq`lcf z=BnxSR&m1*S>QJo5(p74GNX;j%KzR}G$e9TiAOKfLl`^;q9Oa$Z1}!5OtE~uk*ttbgi(A@F9O8}KN+WPRsMvO1p0C%aQV|6 zl5lP)5xrD`z*6$CeNt9-_5^LqMt>a>P;wJ%4mKxPqZ}==2gh4dP>vb22TVB!k|C>v z5t2!|!8+wiIveY^tD|x;i1(q~CaC+q-IEKFSCk4yEY1dsYEdou;OEbQDk4A64;57V zDXvLPQOMGGgQp30553p(a}iHHq7IU6-<(+JLRt!S`b?!R@)2st{jaycR~!v8s2~67 z)TRM?CVeB!`u-N87_(u8mq$vY@{^1D_&O+={9A6&Bo_BIb05wL9}Ho7Becq?U+wfR zU_*@p{5=bJ7GGVZH7x|wRBjDJgn8V2f|gbqOWw^3v>{}L+MviBllnlZ@IJIvVA$ZC93GFa0e3UhzhpgIWqc&yU&yN z@;t3$`MwBabC}mQKg;-+>lg>vMo2ls|j}Vj&+%u1Mh$p+WJC{^)3B0QG`SuTWI!iL+HmYe&{*VOSjyCUCJCE zKfo`~(y+z?vsgShB%jZlDr_+dc4gmKVK+c0bJ0Y}?(7aLP$_L)v^4&3mChw+*&_|D zt<s=sWki_y`aYfns)o$!_yTK=FD?lb`4-;NT$DI<*{@DEP3b7axw2 zNP}T79TY>?H+HFZ)0<;|{raz$vxwd5#3AKx@)(l+`uzbzvWnuY3~DWxt#2|w4VYRNylyf{gM0OU^gNiZkj=ld-wX_ z4tJyuVb|VdxU${Pu$J{NxnOvhJU|EHVf_T-bFY8x^b{+QvgW-93ESZx5P#WUi@hXc zpy&(CqvPEofCO#ap^~lq2V!tzJdxy%ls5J%B&>2aeaF{11K=;=IL8R#k^##r0#v;-o=v z+>Va7q(PmYFD4pu>ze5UZwp1PS|T-4|29)(=cMKz-hD23U8M>}0`k+6vIQn{mK(tRm_JQKe5h{Hmu9{^TaOT|tsfjUF*A-+su5pd zTeP5&9jn}~{fz`w1(r+0*C=EBFL9?z=W34}{Fb%;CwqhM*x^q(^lYKrtgyxZauf4i ztQ`hD-%ZqHoIQt_?gr3eY3ht01A+_t8@~86z3eW(Jk2OxzA6ZmNjo5 zF9FWWE?@?}vK8L~KG6u>>T>>#=PZ@qaR+7cvbh@tA|FtJgb6eShVVivw17YW*0;w1 zwikiVRU|4gK0BXb4qEwe6~-c?-if1DDg-68Z0kl4&LLEXivpM{#})k(TmzmUO~*i9 zP$}_9k$vF{(wU{83_WiFDD#hJSU_GTfR`6>#*3f0)D(ksym{!9pqI z5avK;o37M^`_huW8u|U1mJphIfqu1M>dwDI2yuNJ>+&bY%+5^?E-M#GK={o(Uc((@ zZ?`j{oVdd2002*WY`8-8x~tSz6*rt%I8Zu4r&1BxiUQ%vsUVwEMBpZnNO*{@&qhg% z>CtCzq|((zy-9r^3segu20hrKU)g%m;Yri;yR5^wX5kG9PgAEj@#$U5G2#-nSm2vTMKwZgL56vdRaLsu2uz{j#jf{ILGO0!$v^*}WvQzvBvoh@HazaOmK>k#2kWP)5=udBck&;v1 zD}%#cBzUBsjtzh0FA3d483A12u*aF4#FxkxK$%8Cd8oW4t143Vac?|7P#nq}Bk000 zMtmsJsZ*5s1`&e$t@%?vf3Vlr2)#S1tcbztjn_PTH_qV&$6ekV6zQ6vq|iN(&LIA? z-$XfA#d@odH|VrVxl&}{;}sY1jt3PgTNP?q0{28kyz0NA+6)fuT%|~C{u+Lv!&KR& zB3DKD=F)OhK|Hi)D#wo63Ra7G*|l|wQ`~1nefr-{u2cK{aaO`Vb#8d4NVA(?ys1V(ggegH zm@<}rj9F+DUMU6SvxrFvpk}L)H*sk-O!7*qg}zT0_=lQ{p(Pr>41es?Tc+f$0J+&r zw3>NbIt+j{Mc@eI#HIuDEFF;D(BVOWCR>@`(4iw!RVuDK*KvjyoU2sPJ^BXm&t0gEuDZ&awV(f3AD8;ZVRO~BVTw6H$`9VE z_xzo8H*{a3;*=_vzpJ8qdMT-v5rc?(m7IoNO8ypUMf-^-50yP3WpW$!rj0{~#4BX_ zjF?3#O(6K3sUAc~)7PF+a8XbQYyD_nmX8tx08{ByT)ktJAgB-n4B`FDn%>c z0AQ#x!qF*B3brdW;%mNK!;?>0A_B%ten&=fO2Pet+{iE?BfBM|D;qE}a@x8RG79tK z%l?9Iqp48|o~m8{jG?xtjZ?TW<6Tp6but~7;nm(urW$iv6XtHf_$BZD&p27onX6>& zry%g=rm+NO6Eg;juZItbA&jl4rz{`RWL1bMJ`@H_mj-Ms4%HOR)VAG`4F76O{lQDy z%3{-q(gHe~mSU$6*!0PPA>(zsn*mG~e(liXPtW{Or)Dj5#&jw(*=!!%C&oM0PC;(w zD>YvlJ78Cv@3_o-Bv^?%z4;#6LYeu2;jjG(LWyQ>SDutTWIyqJVO zYExK54p&Xo8!EspwCIfzORGg@=~Cj1tdM-zn9gEZH^g2?@`e4t$Fq2K3Av6swtRAh zrY1?2kje^oc)h|y!GF80HIoazzta#_G=#v!U&o0Uw(?XC{!||SA20fs*>hk?EF>!EK4i)unou`&&JR9o69UoTKB_~beOme zjNVwqOuf3qN^09SoZTS-{d#|1lZ&1a+s5#=4xT>mfBH?4$M)sLo7?0s)%l-IL)Z<{ zs?7V=r7Y61c;u~Wvw?&ayY*N%)nIPNJ2$)3ifr9OgmJ71W)!v~wLt>R4L=!Lh<$+| z7~gFJH?85k?Y1DF*cKNnzS%E49B$O7?n*adUAHT2N@8fqWpNlguj69CY4u3>B;0B&&$U zxE75QePf4GR%gZh817)HvI!R8%r{3K97y;bZ_ejW$|cNa`2IMu4;jb;rV9FrA5s5u zSaYo>M-63Ng17GIwJ~)3b0yL&Dz8%V8*mIUR)KSItBCo;V-Q7I`n=)t>EyGD2xlpx z^`F=~=TU;SyX1vv^TL@}&`80me~cuU-D5*w=SzJEIJh(_pbo=wZQP%FC=>u zmoWb(k03RCe;C;_&3)CFTPz7%sK9-jQteifK8`yR$*1n7MD6#&%xu00aQWbKH`fD%3+VNRd4nCHgor3d`cVe^5QtRdfQETqUA zjJz)*-kc``K8geZM_z}fk3BUebGMvej9#|S88G*wkV$?`k#wl`svBZ;deAzWs`#34 zIh6VK>b}0Fz~MU|@`zO8$8328y<4k9^XRs~tgQT6h)45e)?;3NcYhLq_zRlC@c}a$ zN>#M{9Z4-l?AYoT*#Quk<^G=dZrJ84tg0WpkWnVA#k;D3;BkK0bl|$&p8fz*yhCyVA0dTzLu1hJdy^ic^M2o(X9x7?9Ysw z4}jeN1r`P2??v(cO1*8sCMJ3W6}5>F)b^C&EiWTf8p)$bG+?R!ei+|qTPOZIqBRT} zDn5KHH+%<3swOq~}$ z^dLrpcSwkOsbem29mb*^w5@kqeq9l&x8wS1v{*a^z`^S2Erc|d9SvQg$7VeS$@YFn zdW+)ty8Lce0C0SB4Jb#32_;O5q>)*NT}T?feqG*_vk64p0_efi<9?~cZ=PJ(F#@%B z$t7Q3XZ%NMo2c=xRFT8(!Q(243*Oh;_WBP(F902w$+~k&7=>@C`l{(hNJJ9780veX z=kAaH4Dk^8nzhy|9~pW+7_lg{5%4-dTS=6Sjh@V+5=q+|u!*VTd8i!~{Ecb5s8872 z+@_=}ms}x+N9`9-Z5{>e;o4d?3znX#kxcFA*muGz3Zfm~6^y2kE=afUe}KMh#5$k@ z-1Z0KZ;c82@m;xMKhsX|A0O4I2yNr}Vi94$ZVok9;ZW-laYrz1e?8(3 z00!Jo#&$=}X|w&ro88Y*c0_^6h@$!9+;LAG_jWO@-IftXeh| z8&UvWLh(nqw6gY0SWrAE1AhRopBcFHcY_2oDkB)*-M>4${Q|cb5EiZ=I8NXud9DgW z{F6)tn_{+)XBS6+xnZEEh0DhWtCxIog_yz5EgV6KtFb~XtJ|8-gM24iuItqS56>3R2M*igzW%_S ztd0+mOi~|sI47J!$N~d7M7rpKEtu^vln|-tA~76AmOTq9Gii9-*##y_Tuy3y~Bs_DH0#*Ykpoxr~hvL z{0#o7evz}y1#R|oO-vK>DcxIfN4Z~-ci2{@-;6rb|H|(D_ZwP5;mv*sH&6Z6DYPi2 zcsMlfkKV`C12O?V$;J^l@aK=mV{`-TUp;;ZfY^d*^t71pSMsWV)ZaqJ2Q&ugK3P&g zR|L*GyK9BBot6_YhF`Iki=O>zjnn~~TUA{Y?6vG(={X5FRiIvfxR83iwxWq_I19MC zdqA~ZvP;iLz<(@(<1^5QyOBJRBSpc7Ka;>(6mru_R0YebXLS&wKQkDDL=+y1EX2JE zZlnT%-Lol3PHJ0E-C@@Z%7~wVrF^B}I7X0v@1BSQEvsnCAMtb};HA?CfN-Y)Zyl_LZ+4i79e8FM_j0?F zuIq65HY2t!@xXs5{|w(XD><-DX?Ex{1nl`WZS$~*%Jq!lDFhi#U>d?{FW2tD4tTa9 zk9Xr+gfc9thy-d1W!EEkB^;xKy($|`XCe`k0Tc0fi{J76tIaaV&XL(ZXmJ5yF?bz^78lm5QW$eCH{kYW(;Uc0rn2NYB_BM=sdO zi%$h;1mp07N6Je=6*szE=!uzpZn^PBn8pbjog2eI&S3)XN^d}ZnOIzD(7TkUZ5J#4 zu~Us#9hc;5#RUVmZjD!K+t(rOsthsf&G4GX2ei+}8L9DcWS|8JfE|seKa^m0bnM4O z=O>ivlqOPir_GSAZHvGYB`oKs_4$)L={W+;u^?fypNOl!=yF_XshV=jz8&pwq1_Vk zb(a7WtlLc}gXAv)HAfv5QF2qYCrKR!=-O`)o18>tKw-z%p|1(9iR>B_x`}QXHGV%# zGiB1F(Xd%tnzrM_!jXXz$2YOj!?H<2)GPBhsRVvp;yL#GqvhbplQw&|0~wHnlR^=;B0oK$~cet%MUg}uH=;d4KUk{&5c;v8ba(@XU* z>b3XL1CYYKFePh9ZzM5)~fbmVynbcg5u=R4ZjBL%nA ziElV{)jv+{FS?0Hs=gkQ;mXr!IIb*Ur#Ti{xI@>j51Z(@ebYTb(&h>~c}Us1`AU2(t{6q*F*wtI(j<1iCWisW}uCk*3E%*EiL@H&i{M8J!~bWo?^hOn&DKL8 zd`t&K9fLbn~+uUTu5ju!ZtAarVh6=YOam!rhKTwhBynPXBn80ARVo$psT>+`fCxduAP z5zVVIU!Zykme&FGfXNnafS>o)$_@q9fL5GVyu)RnH_CO}HiuRzX)ZbRrF z{7Ps2XXH0`hsH5Q(i^v7SX}#oed%I=D_mt~!8iebOc@KD!e7BX@<&tng~gxE@kjq6 zaQT&6^#F-N9;LOpu4VCS*Uv)3kE8}X8;ckxj2Gp3@s6}@7n4MBAQvl}=q1?BvpBpI zmQaTmeVIjn^U)hNg#R@3O0lDtG%~=0fVTYOU>^`6W_iPuLaWrohk5MjD%dUc&;3Ih z0RYruc*)#UK7W>qOEY4ypGTk)>@eIhQI_9dc?>|W%GA`xN6pQcUqXc+We;MphJu!( z!iwLQxaOt%PCzzF%GsddbSPDbJ$bo2l(M5-*RMJAoQPYEQAv>W)lj*O+(E^=haj=a z9kyV?bh#T8%lRsMS-Bfjkr=Uq4;n8EDpbY0Qas*dH3u9dBmsQ|7-pm9IHbN&3r%z6 zA(J{4JgobZJ5kRN&erbx0mPXr##s6l!de4qa~>oY;#FO-na*!4={opSGQ1-gYSlBT zFy8-$+m#5Z&E$x~u1s?tvk8C)b;D)VlP#`Wy(f_wHytIpN_Z1BcxF)krd z!{ZXCx&|J_sD>bdl+qI+tAngY*TP>bIOMT*5}%F5Rp49wy7DomIc{EIT$9g83KYR* zS3VG*Li^(_rgKdt1}k1)Y^_C7P_rt%$hkly8P{9QJxxn20>TVul3EM8$If0I$Dr3L z*sgU!zZFJxZFs`mRBZ=u#sVSgciR2?8Ud6?0&$n2nfY}*XAGG6L>ezlA*^Fi*Q+9) zQUm&#zyD4mg8T+vLESfyP_062Qy+1Ignm3}$r2tmtY)UfnK|?$bCsbH7+uk$5rnIK zP)975&GCZ8y^*ht*S}%cXxDF>PpV+7H4l{}x6M0|2Pnu4Ux^zBGXQ%?Lp1f5yrzB> zEnDN`8UY@9#NOczwoJrN(tM$Q6uMag_$GY5viPMNtV#Dj@*s1x|- zH8m!MK0<(-w!CKj0-4PrX9=67xs)IIxheqPf21Y=R?P+>WoA9Q@=bTe4>sG!BMxtY zz_Q#zm3)kskxRtntk+sZzSG1=&J_IrcB6ex~95&k7qLoA$p2>@9 zk^gE3xy5ggo%GQX3==Z4f_0$f49L#i{+Xz~k>vHLdIS90I4%!3D~M`Vf2rMYUzaw{ z;(M*7+w0pVqQoQMJ)fwqLk7d5&fBI+CfGrz5anTVfj>{tA>V3ohokFVc+e573S=Iy zQbNy;lvk06)R9xYTUIs@A68hKIQHg_Y8nL>%XI8^e_`p#E-C>(>=4>5NakI-ux2r6fKg>pQzCfc^I9QE#2|?GPUQ`Yzl2L%Lv# zbM*?m!-oj^X+`37BW=)<51`w<>E3CkL;+G|y;9zFh! zh1>6IDr@qddzM~3y(s84Gn9zAR(i3)M;zhNg%KFredPu1`!^pr?jL>28mxLt{~<2< zOb@}>4+Y^>^ykqB!UtoY7^(lak7jnI|4XH;K9g1;sb2h0o+?=TOK*(a?wiZo#nSg_ zUs!U$#c)+yljENi$PPv0eh@(NMUe+seD>jv9+G0z+Hfm{k+M!75GT<%pm*hM-{v=H@p}oD)g3}?wi?U00O_*Z*%u{@HAXP&pHLoiVRd}HsRsMxCQ~}Pg(21E`Umu@tb0*7D&YhJ^;%e#3CwD9`c+VdCpo%L)jC%t-L zDK~xT$`20)kgFD%ENxE2c#*P}8FN{?3ERO|v|`S`Zf#j99p~Q2TfmG346_d}wCK_M z-Zfhx;p(Hw<+SCLhvWUG<^X`#_(*~xFNS9*~ zFXgT=Dqw*2MM{`iF%8^gB6x9MRAsa2^5(o5m!zHw;7fwX)!$O@nxLm~<7v*A)oT*n zJ!@&#R!nfZm%m}@V$V37Wk_-jj`EniuF9HC3g|LktBTV;cPUzCPG01uqh;gOS&~RY z%h@c1;=;NJ6|Y#5&XJKt8meM1n?r@Q$wrzFw+NQ(fUX^`Q*jwo_QKz3H+t+ zV_Ch0)v61tHJR6ZTT_OXH^^D$uc^sBKWQJsnxm~kND$8``%cx>QdfZ>e)W?i%0xz-84to z(7Rb#2oJhsbsj|9p)X~VqhsEI3fKg0@cU+QFQG1_wd0^V5b=4iK5$q#e1g@Rv+m0$3S^^0j8fKUv~P*Fy^W??KC#X zE`J98hR=K1XaM(SKod@$J`C>dCjk!Sn9!AU(gms&Tp0&Ftw0%-W zN5n!I6alsO)@0u1zW%m`i@ifv1{>A*1<<_riVH=5j}+68YkB|i-w!%%$0M`i9|j)k zwR$svG4m&iJ-egm$$?c4o|JQ7c|i~r1h+_%uLWZ(r81fv`rRka_*^GXZ{ zSW5s!?!y)Z-KNJ<5aV$#7xg2o5 z^S|@)6jpfLIH+F(dUA zc=#$>SAID0Ew2CUh(vjhW#1Y3y~r($DRf8o`f0 zn|#NYoGUupi3QD}+ zp67*&vFiQe^FwoewFD+1Zmt7|2lREy z{H3xI>C!a<{8VLL0>9|;VME}6=L4AwtDKsV^TEC{iqLi2ZNB5i5b!jrPf{M#%oSHn zlUKS*ZgJYy<8?YD!)Gcdk52;QDsP!#zlx3H2en*004jg z8UK9%!~y_SlU-_qKoG|pf&?oPBm^rGdYQiDLb+1NG4k3=1PLKXksu|A=wfGQen$H+ z-C?@*fsyppnB$Y}nrM5DA${9-S=wms5?9@A-Qi05_QfKKmhUMqIqu+q# z@pCyZQg@rVmGq@lYaDYP#T~5t@7zNAoALJ#up=wI_iJ;nw{Ph2;5k&2=S{#|npZzl z+#8ympMS`o_2Gy$2W^GS4El_CR@jM=(E-;%6(YsUD!|FG--6$mG_|y3B@VuHX5Cv@ zV{buBH8xtR1f%3$V7gRC%$go}dlKJ;h=-Y$khSg572v7EZw~jk!Y%wnsjFY3Ij67i zTTU8YIVNZ*;7Gx|+e~#V{`=8Pn`x!5XIAn%)~nbFx)bgeWZF&{sGr*`i+ZNF;%-CV zkYB-@fYa+6KOxmWeg+9S#O!oH-@r4z@+R)&x8;B1r_q2P@rIJ~(6z_!_qXx<10@3h zSno3_-+d|$(~8&hLbF{IZX;sqU{KPFn{}*Gge+WDezJ65 z-Tfaa2XTr8=_;2N)uUZ9Zd*z_nb<1p6?dWCB`g_7no?PAuD1%aEl;Y`L`-AS^0al= zxb+otQF;n4*TT~G$+Cr>sKawQ7U*iNl}@Z(A?xu~Iu+2y-&8BQZ76IFUYdYfs>1Z@ zp@nRyXN&rp-2Uv92cl+TYKLlix8EAJmTsdpQt0_}dcxgatk-F=#WX8dLCPy+cIMYg zuSJwB!EM_+)%t^Z_P;#JnsxfIM_I84@eGamTR=cjy6FJv+ zwp|)4pF!NBR0fC^oUu*QsIfqtrc@S!8UMVcD`_8UnjX-T57cs?mKSK}M2oJ`R{9nK ztMw%)Eq&YO+jfB2cdJc_?T1`?0#G8Alo_Fp!TGRm zBlHvX^*l{eZ!vv4FR=+>6GX%o5ph(~$;$VyT0P7~QAr)v@n47f`{!T(e*fpsKfZP8 z>+_JlzBc_R`?2lETR#^4DEhG**PlQ7!P#Mct^4ubk7@sZ@5haE!*|_}w*Q}ube|XY zF5D@hb?(0_Xw>kWj(4A*@ape8PyPC`pBwmWSwC>Em*shd)(#mi01o@B1d`901YkTg$VVjR}U;Z)M zt)S6mncVgV08YM7`(49cg(o?^sgeCHe(FdYkMZo$wo#N-7!myjzIG+&&R zP0-)QC%!$wcZw_v_AbmvFn1h%YGd&~_|!Ssw+*p1!;Z6H>m%|)(`_RD2Xw!J!&k6H zon>-=U_0|k5oB8@&#}rcwx`&;V2^Y5zsX$?tNV;QlkFa4m=ZHBdtLI_S3G%sFH_Eg z@7Nx`c82(}GpYmRY@<^KTjL+In_}P;u!TQNmMLejv7Hl!n(^Q913h6$xzyyD=eRYR z9T=^$zQ(Tt>ks^L3;eF=LteN4IX0N4Tpy0eAj~M|hdN}kM7-3&-=bgVST;I}r%y0- z029aL`QB(qrYhx{cpC2*-#L8czg3QT!+zkY@SWH%{ zl2-{;R8?nN+ojxfbjdT4oz*#Y0$AA2Z?KcPDn*+z*L(76X|_B?hpK?eevspKdO|wOFB!#N*yx7 z23ds87icb$S9ame`3vPu#)RAXGoYKwYj#rRq-se|_nA73GUYC`17mbnURm+0x#g8!7^XDMgp8T^|N#gm*B)LnZg{+8`->xwBlEyc1|QL2DrULJ9Bs(hWiLr^8PyqpuNCuw zULvZ_y0ZW?k01}7mUqOLeAYXK63l>OzfYHFwtS@AEf)ck`!Lh}m`y^AdoCd-nqAM> z;lMss+wKsfUKNO4=N)pcze3s&b2a>5DR-Q8kWn~N-YK(gMoalYFC$(C@{c+PuRQ7y z!IJZ#6ZcN%4&N`?A0AKYbX#m8y<<14ID91U1dM(S-^YkGt1Fk3bF<=HFO3`aIrBZu zG2yS5Huwu)c+Jsk@`_!m5bIZT+ku1iIIC+kIjZD&+&fskj>*OT!V*r7$vdZK_Y3C? z0Ko6{3IJLE#BE1SnugF51zRnu)f9B=a*DuK(ttHRuyws`TY4HkQrf4KIn4d6>@h(N z#nYms)T^OQHLf2{x*~5*g?odc4a)9|ZOX+`O1ZQ)&pi%u*<>0GQClx|{IC6DE@Ni; z+Fu^&SFx;JE-nF1D*V<4yq37O4HawSO8diGCaERJ!oBsk5b7Zi#Q& z0^?SQc3-!Z*)K0pdX!YQ+(r zHC7_7)Je3>MXMP}O8wcY_BPnQWiKwHRT_p_=CgG^z7^gsg+ods-A>~*o~B(NDffA+ z4^eX06q=(JR;rOodLmrzgZM|PbqF)7Y6?Wza*TMWp|;8`OM5yPTgrf$?5dr4Eptwa zq+Eu@G@q{(dT4=;I*DGkf?1Keg$6KBRJs7*TR_%{sT~Hh2Wb0jQf{A>&!JvrZcjlx zBjjd>IZ36x#4M9UthvrLF^htv$$g=6STT^dhQ#@rWpq9qtBQ*t_o*_O}Omb(3B=;dtd zu~{C(Rd%7iKd*62njF?H>2%bxwXH6{Wg1o1wxMCEA zJl=lkcMYW#PG}3+y6RI}rc{fd4F`3@Sz7Af4$XHNv=xtRZB|=GQ3nZmOSB!%?D=}T zsoz@B_6e#zBJYPtYa`q8iQB5GEl6lFaB7ZMI-`_bu@ydOk-7_}T27QzFRq=tYe7$I z?NfG&ETJyWZy!$T&{K-y6;+^hx2u&w+Lzq+{Fj5S%`5Av zbz5QImV>)Fugw~032kZs;KNN7TDyIHYL82b=hrH=lo_Ob$F1{NJ4V(@fW3X89xv+m zdpq1!R{B(3-i6sOV?}K`)^hk)b*P=V=#yrv5K)50x0SBNPOUdYS{+^v+;q6DeU+{o zQu;qyg2`6Nw7pMi;b7Z|*iwUe+^qGiYwK^jqT6Q6RLQx09j#$i3KY{)Co=~@bfcKNCm00etQ8XE4(>GyZ8m58Vx zl?z1NW|tMovIQEelSHgXQYPls`LE>`D37ihTv+nc^eNWOvahpod$my(g;Y$+eZlF$ zQF<-jZtye&)lD5K`Fo0qUWVp$tg`)~ZGC;av{U+k^d`Xu1UGeLY8&n*nw1-%ChzD$ zt(x9aZl;tWxcwXMV@&E8QeK$$=cVmNsZ&)Uc-=1+0Bh%cs$D!PF(uWf$^6 zWU3uvzQr`eue||voqGBgY3jnX4%P)is|8Y8Jyc%aH`~4CM5=dQ%jMBCUsjl_Ew!%^ zc`Bh?y5o1RxD8Kg_kJ1`X%At?7r>%&s&t<})D^{XhG zZdy~>s_(Q!%vGo;wA zpeJXR9St$7PGZ8)sUUo>PSq)f_XvP!D93M~6g}k@i{)9_IYSej-zg)aHbV~Fe53)Z zfoECv)~EwnhZJpz*%w z=H=#Tp+y?btIw!)g0&_*X_7=Li{D&{lG!}hc#M?A6KC^HVsM_81ThDc!6s^Y1od13 zJU`Q<>ZXZ^C+R-ib({b_XYp|`vw31{RRL3{(t{{|3v`+fOJ=- zPA~S7C{rYh(ct++6S6#K4D zQkNhKWYX_HxaS=}SR@rl&&g>3!{WG$AilHlxGT@vvyQ8XkbZ~*SEvpTpvZ=TEK$Gk zeE_Th09}ug#L0bZ3fCDmI@F~EIS$O?5tz>`vG6CG z!Xv2AqZz^iUzsH4JYrV3$*A!`P^puVgGY#mk%^gwm5rT)lZ%^2iX*usGp{l;uSAM7 zxi~j5CkMo2Nd}SZ$w`^%AO>4x>gF%u9o12GE_vjQ<25VJ!uCloV)ut7tD6*C8r z;N`gLF2uo6FHj?K{~e2!5XUulAzqHJ{}}~7{AZL{bGw_}N&=)rnB(rz{|9|zLUtqal-X(6p6_zBQU$6|Qs5FbFfcL7F-S2j z3UDzi_RaA!4^1^F6k@NeG;+^$_q9?4TBRhz#baP#Y?79p9N`=2X)MH3Sz&Az>KI~S zWbB*^^yyW1AaTvziW5j^v2Zc{(g!lHf|>pD4io$U2(<$c0T_@@A_4#mg7gCePf|k@ D00_eh literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/char_varchar_udf.txt b/sql/hive/src/test/resources/data/files/char_varchar_udf.txt new file mode 100644 index 0000000000000..570078d90e65b --- /dev/null +++ b/sql/hive/src/test/resources/data/files/char_varchar_udf.txt @@ -0,0 +1 @@ +47.3224 2923.29346 diff --git a/sql/hive/src/test/resources/data/files/datatypes.txt b/sql/hive/src/test/resources/data/files/datatypes.txt index 10daa1b11c26b..0228a27e927bb 100644 --- a/sql/hive/src/test/resources/data/files/datatypes.txt +++ b/sql/hive/src/test/resources/data/files/datatypes.txt @@ -1,3 +1,3 @@ -\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N --1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N -1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123 +\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N +-1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N\N +1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123abc123X'01FF' diff --git a/sql/hive/src/test/resources/data/files/decimal.txt b/sql/hive/src/test/resources/data/files/decimal.txt new file mode 100644 index 0000000000000..28800f5278b10 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/decimal.txt @@ -0,0 +1,18 @@ +55.33 +44.2 +435.33 +324.33 +324.33 +44.2 +55.3 +55.3 +0.0 + +66.4 +23.22 +-87.2 + +33.44 +55.3 +435.331 +-0.342 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/dept.txt b/sql/hive/src/test/resources/data/files/dept.txt new file mode 100644 index 0000000000000..292bee6ee0370 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/dept.txt @@ -0,0 +1,4 @@ +31|sales +33|engineering +34|clerical +35|marketing diff --git a/sql/hive/src/test/resources/data/files/emp.txt b/sql/hive/src/test/resources/data/files/emp.txt new file mode 100644 index 0000000000000..a0e76b90e57dc --- /dev/null +++ b/sql/hive/src/test/resources/data/files/emp.txt @@ -0,0 +1,6 @@ +Rafferty|31 +Jones|33 +Steinberg|33 +Robinson|34 +Smith|34 +John| diff --git a/sql/hive/src/test/resources/data/files/exported_table/_metadata b/sql/hive/src/test/resources/data/files/exported_table/_metadata new file mode 100644 index 0000000000000..81fbf63a54980 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/exported_table/_metadata @@ -0,0 +1 @@ +{"partitions":[],"table":"{\"1\":{\"str\":\"j1_41\"},\"2\":{\"str\":\"default\"},\"3\":{\"str\":\"johndee\"},\"4\":{\"i32\":1371900915},\"5\":{\"i32\":0},\"6\":{\"i32\":0},\"7\":{\"rec\":{\"1\":{\"lst\":[\"rec\",2,{\"1\":{\"str\":\"a\"},\"2\":{\"str\":\"string\"}},{\"1\":{\"str\":\"b\"},\"2\":{\"str\":\"int\"}}]},\"2\":{\"str\":\"hdfs://hivebase01:8020/user/hive/warehouse/j1_41\"},\"3\":{\"str\":\"org.apache.hadoop.mapred.TextInputFormat\"},\"4\":{\"str\":\"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat\"},\"5\":{\"tf\":0},\"6\":{\"i32\":-1},\"7\":{\"rec\":{\"2\":{\"str\":\"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe\"},\"3\":{\"map\":[\"str\",\"str\",2,{\"serialization.format\":\",\",\"field.delim\":\",\"}]}}},\"8\":{\"lst\":[\"str\",0]},\"9\":{\"lst\":[\"rec\",0]},\"10\":{\"map\":[\"str\",\"str\",0,{}]}}},\"8\":{\"lst\":[\"rec\",0]},\"9\":{\"map\":[\"str\",\"str\",1,{\"transient_lastDdlTime\":\"1371900931\"}]},\"12\":{\"str\":\"MANAGED_TABLE\"}}","version":"0.1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/exported_table/data/data b/sql/hive/src/test/resources/data/files/exported_table/data/data new file mode 100644 index 0000000000000..40a75acfa0016 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/exported_table/data/data @@ -0,0 +1,2 @@ +johndee,1 +burks,2 diff --git a/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt b/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt new file mode 100644 index 0000000000000..6a3906944cbd1 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt @@ -0,0 +1,3 @@ +12 jason +13 steven +15 joe \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/futurama_episodes.avro b/sql/hive/src/test/resources/data/files/futurama_episodes.avro new file mode 100644 index 0000000000000000000000000000000000000000..c08b97b46e3f1334087b6563dc766060361e43b2 GIT binary patch literal 3044 zcmaJ@fomLD8Sjgslu}A5M<`1O@0OV6LK4#SD5Vs+-Ay;U%_doP_nNz&if?D$?!3#J z_r~|$Y`Q%wA|fIpA|fK9A}S(s93qE^$e|)4B65g`h<}T}Z)S6(J$nP2$;_MizTfxz ze&6rSz)hGvS03{?%`J$DvztCjWGddDd#f6znXKNaIvH6PSJ$gXq;x&XL+(T>aA%~X zj(zpc8*iO1Dtd7CqB5cSWc`CBFNL?}xyv$B-p2B|>p=y5X4MC$%((jGQ%~j>f|C=Y zHVNOnicc>I#d%^JPU?04)v_Ivh6_9ymd_80b7_BZx-_CEUu`zQMs z`#1X!`!73%OOWX|cY4hKpX2<~$AvS;#b=I7&pj@mJwBBFl&ufbwzB%ny|b5R|0kB8 zy{;~=%vgVFh4YpDDIX+G zB3W+ydX3lB7%(Kj;GtdNhGHyNTT*M@_WUp;(gDHH22bI%8Yh7V%X>1F=}@|*eZ=Ii z8a0v5t1w+YN~Ga6t)%h17J`V5H=AzCS2!Z^zRq3Bn<5I z@Cp%kyPF68y3Jh~$QlkJ`iaeTEZs^+dXGZEz~IF4dlRL7r6-VnjCg2-+a{u-X{2rD zc|BJ;@R4=&bhl+s5Ywe-Tu**BSD*oP$e6d3=ey1#Ij@6iLe8 z8SmRXB(qd}R|e1bMJ%^zz3c2y%aoreJkhmTror^JeG48O)UYF_+cgsstL=D7q1z@% zCqc#Pjh7ZEK+BqAa=0TX1TDpTytQ#ePE4x}H?>AZ{K%%lEOvzxz>iW$UR$+PJO)!( zLsMtm?^_#Jh!hRZ3Tp3C9N;KI-xCfv(G?rxZ=R`mliEwVwL=eR0qsP%@M2M2UAY+I zDu73FqalJ5k5@W*bWFDhE7aql;$l7;Q%DX4%sSWG_?GmNV*L&8Vko-$kn%MIrTtfW=p^N5`MEb%6 z;h)F+_Df5oIgS}d(CAo(YDY>fz(NaLo>-^jbxewrMTzx%JQf;5WktGHk`_P z)rJEh;G*QYWkcmY_G9m%hPrUa1>-#kvnaX-MrbaOb0E-1daVf?iEVmoPr*4zw0nk% zjrpq?*gUWyN)~Cdh)SfF&fXnJX<#rI)&K@e_{2)?NKeH;&@30hr&mc>{eUK|(|=9j zj0bl#2ViJuAK^ARSt~cOLt3)3x}Ew3n5%(iNIO?Z36y)~aOka;LFy?Y4RZ|@4a(u3 z`2Zm%`#`Do1mQs9Kx#R`L|b#e5IHV1!9`(qqeat z;#h3#h>fC4TcsVI^1Y`tCYjf=Y!xG(+0r&Jz9VJGNQ-=VvhhHfNbv@J(R5@;1bo@} zOh#lf1uY$@7n)JfGrzw{H7^y?-2q*EPlU%WqGxcP=os7`^FycLYejRr7%b?a@I7Ke z-0e6=Uunpw1zd+8#2BN|2tqZH^u@%Mohz&F$%)m}pI3_h%)aNKajjzbL7!lXD965Y z~51lDNq-~p<)<}zmB$x7+(2@cZVv6RVs`OZAKz3t!g|*S_6NQd;s!hgYeL*7&bGm zG^~y}T+=D+x0OTeaIk-{sJyK7B-#RkMX_g51nj%M08Sw4QhKls#md4MJ`k zD%6B6|Hk%ekvLIuM5*gyoB-~%ha$@W)r^NC6kV$40k-*OZjSjqv?2;G(|djOi0?x} zP{C|#=lj-NYvzV@i_d`ERB1l>5gwump*AfY;4@)kM+%J45e8)Gftm<#fGzxi9Ah`V Wc>^0q1LWnPsc9cwIS}PH`u_kdwz-4= literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt new file mode 100644 index 0000000000000..c242b42b6070b --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt @@ -0,0 +1,8 @@ +name message 0 +steven hive 1 +dave oozie 2 +xifa phd 3 +chuan hadoop 4 +shanyu senior 5 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt new file mode 100644 index 0000000000000..d5db38d168be7 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt @@ -0,0 +1,8 @@ +name message 0 +steven2 hive 11 +dave2 oozie 12 +xifa2 phd 13 +chuan2 hadoop 14 +shanyu2 senior 15 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt new file mode 100644 index 0000000000000..f7a763d8b9638 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt @@ -0,0 +1,4 @@ +name message 0 +david3 oozie 22 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt new file mode 100644 index 0000000000000..c242b42b6070b --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt @@ -0,0 +1,8 @@ +name message 0 +steven hive 1 +dave oozie 2 +xifa phd 3 +chuan hadoop 4 +shanyu senior 5 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt new file mode 100644 index 0000000000000..d5db38d168be7 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt @@ -0,0 +1,8 @@ +name message 0 +steven2 hive 11 +dave2 oozie 12 +xifa2 phd 13 +chuan2 hadoop 14 +shanyu2 senior 15 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt new file mode 100644 index 0000000000000..f7a763d8b9638 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt @@ -0,0 +1,4 @@ +name message 0 +david3 oozie 22 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d b/sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt similarity index 100% rename from sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d rename to sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt diff --git a/sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 b/sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.txt similarity index 100% rename from sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 rename to sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.txt diff --git a/sql/hive/src/test/resources/data/files/input.txt b/sql/hive/src/test/resources/data/files/input.txt new file mode 100644 index 0000000000000..caea9919d1d7e --- /dev/null +++ b/sql/hive/src/test/resources/data/files/input.txt @@ -0,0 +1,7 @@ +a b c d e f g +a b c d e f g +a b c d e f g + d e f g +a b c d +a e f g +a d g diff --git a/sql/hive/src/test/resources/data/files/keystore.jks b/sql/hive/src/test/resources/data/files/keystore.jks new file mode 100644 index 0000000000000000000000000000000000000000..469d8a543a4d2a94535996ceee2a24891699b1cc GIT binary patch literal 2248 zcmchYc|6qn8pmhGSSK8N#uk#D-xzBsOBlPcWJ%eYvKt!9bZ>*9AskZ?hGZ*ZEJK!L zJ+ft=bdIU1kg`T0##XM*z4vv`>vjJ>e>{IY@9*n*UeD|EeO{mE$Lfz&5C{Z03h9p}+ z(#5${Qlrizjd*Qt=3Y;+F6i8J>X;dp-8O4&ppIWly63Y$Su+~i$j^H_U7hsxzU@}* z#OCn(Z zU{7sNAB~VRuL8JrNrK~q#6rAJ*ICkuE4NSp!WkpieK#@#EAOc2TU%3KH%WqHY>Iw)QG#c#aThm7Tvs zW9V+`^31@As*EJH_$HMiqL7iy`dENe#S6DgSk3P=pqMIW_vBXe3MFMW-sg}Jzi)+Xrw6Hc-|R%_>A>}c53Q2M2@-qHKlPY7a< zlJzoLawdWFprzUu>){rvT(HZuXmzU9x}>RMw&QPcdPRv0RGr+2+>V&VwTT7m5JFJj zy@C>A71mRU-lD0d+?tZzIt(5$*4CIS$dF4R##Tb z8;&bvn8cM6q7GJ}_mx9**jK)X=oJNxFy|VpsQD#Qb$7Rll4gw0hb`t;n^YetCe(6< zz@F1x%oip3Pxnfq+%K^L^Lq9oIHlutMfigC9=iArL7IeVJGDS^y8%WP>C=MA8k*pl zZO3S88Z@F4d@RoDQhZ#bWY%Zod+l?Y2M!FavmNeByx%Y6De9#eSYYN=*Ct%jKkhb7 z7!L^ALZ?H2oycfg{#3~Movg?CI)4_uFQ&(bJE^7S z!p`?Kr<~zrICUyg`2dJE6tuapVuU6n*g!|I#g4{t8=AICj!c#7QE^))msIFV8eRdN zb$&0v$VEz2j=%^NAthnEepOgNAVTum$$HBxgUMvytWkID#jx5riKo_E;xY@dvrc|f zq{ipCQf}W$I1DMdki!{zxf%1FY45A{HZ!eHqbPDwbZ(=s9~U~Za~}D;om--oLZN3J|MW(74-Fl_ z00d&Y0T9{n01@I>%mx8NAW)AkU2cGvgIk4!BC&YEU>G|H2u1vIWSMMkP!1tZxA2hg zh}iJ|60ZLvoc|;IM~#lEc!W4iv3>!87z8f>`w3t|P}Gq%BZL9LpDCLV?BDMvep29s z7%u{e0JM;Rjh7X7i@K!LlA3&*^ut`zIhhEmbe33oAAK3c%)f)!n!lvCC z)@~Z5e0Fc6pwaU1a+zb9u&ZgZdBT;tw@;L44^i;spji3A8R=Y=HgK;Oe>LL;^H`w$ zV`6;**W~093NhZ{Wky#Om+zSyecGaoZF@wf9c-fKvT|LYDKvemB%Hfe{a(G=!(r#y zC%i(;zO=}f9HT-rD@jU-*RV!kCHDhEYl`ZaDtG7hb85hWHU+E=n7pwME%H#E4 z48oiGvoGg886I})=dH$psxr{a-C@tS&DgT}(M*Jbj}hF@=A=nxGWLoTgbfS=m&pP$ zfYi}%O7THOp~Cv>`L|#e+4|*qBy#!)?VWN9d&yr#Jld)w5y2p#JskQtuCO@9R;D*< zh|m@29h@W{q_REwfrDj?r&9uKjv*9#lPxZ+pJtubxw&mJAFuY-&rSVva9edbtL^a` zVC#wxGu$k0QvPtYF=s(WS2IqRkJJ|3lm;_~Z z4D~zPz5hq}zMH7D3D)rDrvl5fhMj@$a;}$*ALPC~b%wQ+J+q%DsH`B3ntZpOqPQ~l z`HgUjJvs7SlUI2_yz)VFX2b=_v1m%>-M%Z+83QV}6oQgs&wZhaV=d1~TPwW~u?lQG z(N<|lUq}mHz1uBT^Xj@`%d(M6ZFgv>=~%lllQ15KC90j(5|wDa!{FY(SAs`3qvi|_ Js_3iE{{opy;Gh5i literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/kv9.txt b/sql/hive/src/test/resources/data/files/kv9.txt new file mode 100644 index 0000000000000..b72475f21bff0 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/kv9.txt @@ -0,0 +1,27 @@ +-4400 4400 +1E+99 0 +1E-99 0 +0 0 +10 10 +23232.23435 2 +2389432.23752 3 +2389432.2375 4 +10.73433 5 +0.333 0 +-0.3 0 +-0.333 0 +1.0 1 +2 2 +3.14 3 +-1.12 -1 +-1.122 -11 +1.12 1 +1.122 1 +124.00 124 +125.2 125 +-1255.49 -1255 +3.14 3 +3.140 4 +0.9999999999999999999999999 1 +-1234567890.1234567890 -1234567890 +1234567890.1234567800 1234567890 diff --git a/sql/hive/src/test/resources/data/files/loc.txt b/sql/hive/src/test/resources/data/files/loc.txt new file mode 100644 index 0000000000000..69910b7649571 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/loc.txt @@ -0,0 +1,8 @@ +OH|31|43201|2001 +IO|32|43202|2001 +CA|35|43809|2001 +FL|33|54342|2001 +UT|35||2001 +CA|35|43809|2001 +|34|40000| +FL|33|54342|2001 diff --git a/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt b/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt new file mode 100644 index 0000000000000..41586d61eda03 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt @@ -0,0 +1 @@ +1|Garçu Kôkaku kidôtai diff --git a/sql/hive/src/test/resources/data/files/orc_create_people.txt b/sql/hive/src/test/resources/data/files/orc_create_people.txt index 884598981a13c..ab93c1400769a 100644 --- a/sql/hive/src/test/resources/data/files/orc_create_people.txt +++ b/sql/hive/src/test/resources/data/files/orc_create_people.txt @@ -1,100 +1,100 @@ -1CelesteBrowning959-3763 Nec, Av.Ca -2RisaYangP.O. Box 292, 8229 Porttitor RoadOr -3VenusSuttonAp #962-8021 Egestas Rd.Ca -4GretchenHarrisonP.O. Box 636, 8734 Magna AvenueOr -5LaniIrwinAp #441-5911 Iaculis, AveCa -6VeraGeorge409-1555 Vel, AveOr -7JessicaMalone286-9779 Aliquam RoadCa -8AnnChapmanAp #504-3915 Placerat RoadOr -9NigelBartlettAp #185-385 Diam StreetCa -10AzaliaJennings5772 Diam St.Or -11PrestonCannonAp #527-8769 Nunc AvenueCa -12AllistairVasquez2562 Odio. St.Or -13ReedHayes5190 Elit StreetCa -14ElaineBarronP.O. Box 840, 8860 Sodales Av.Or -15LydiaHoodP.O. Box 698, 5666 Semper RoadCa -16VanceMaxwell298-3313 Malesuada RoadOr -17KeikoDeleonP.O. Box 732, 5921 Massa. Av.Ca -18DolanKaneAp #906-3606 Ut Rd.Or -19MerrittPerkinsP.O. Box 228, 7090 Egestas StreetCa -20CaseySalazar506-5065 Ut St.Or -21SamsonNoel1370 Ultrices, RoadCa -22ByronWalkerP.O. Box 386, 8324 Tellus AveOr -23PiperSingletonAp #500-3561 Primis St.Ca -24RiaMckinney3080 Dui Rd.Or -25RahimStanley559-9016 Nascetur StreetCa -26ChloeSteeleP.O. Box 766, 1628 Elit StreetOr -27PalomaWardAp #390-3042 Ipsum Rd.Ca -28RoaryShermanAp #409-6549 Metus St.Or -29CalvinBuckner6378 Diam AvenueCa -30CamilleGoodAp #113-8659 Suspendisse St.Or -31SteelAyala5518 Justo St.Ca -32JosiahGilbertAp #149-6651 At, Av.Or -33HamiltonCruz4620 Tellus. AveCa -34ScarletSantos586-1785 Velit. Av.Or -35LewisMcintyre629-6419 Ac Rd.Ca -36ArsenioMejiaP.O. Box 767, 8625 Justo Rd.Or -37VelmaHaley1377 At Rd.Ca -38TatumJennings829-7432 Posuere, RoadOr -39BritanniEaton8811 Morbi StreetCa -40AileenJacobsonP.O. Box 469, 2266 Dui, Rd.Or -41KareemAyala2706 Ridiculus StreetCa -42MaiteRush7592 Neque RoadOr -43SigneVelasquezAp #868-3039 Eget St.Ca -44ZoritaCamachoP.O. Box 651, 3340 Quis Av.Or -45GlennaCurtis953-7965 Enim AveCa -46QuinCortez4898 Ridiculus St.Or -47TalonDaltonP.O. Box 408, 7597 Integer Rd.Ca -48DarrylBlankenshipP.O. Box 771, 1471 Non Rd.Or -49VernonReyesP.O. Box 971, 7009 Vulputate StreetCa -50TallulahHeathP.O. Box 865, 3697 Dis AveOr -51CiaranOlson2721 Et St.Ca -52OrlandoWittP.O. Box 717, 1102 Nulla. Rd.Or -53QuinnRiceAp #647-6627 Tristique AvenueCa -54WyattPickettAp #128-3130 Vel, Rd.Or -55EmeraldCopeland857-5119 Turpis Rd.Ca -56JonasQuinnAp #441-7183 Ligula. StreetOr -57WillaBerg6672 Velit AveCa -58MalikLee998-9208 In StreetOr -59CallieMedina1620 Dui. Rd.Ca -60LukeMasonP.O. Box 143, 2070 Augue Rd.Or -61ShafiraEstrada8824 Ante StreetCa -62ElizabethRutledge315-6510 Sit St.Or -63PandoraLevine357-3596 Nibh. AveCa -64HilelPrince845-1229 Sociosqu Rd.Or -65RinahTorresAp #492-9328 At St.Ca -66YaelHobbsP.O. Box 477, 3896 In StreetOr -67NevadaNashP.O. Box 251, 1914 Tincidunt RoadCa -68MarnyHuffP.O. Box 818, 6086 Ultricies St.Or -69KimberleyMilesAp #893-3685 In RoadCa -70DuncanFullerAp #197-5216 Iaculis StreetOr -71YardleyLeblancP.O. Box 938, 1278 Sit AveCa -72HamishBrewerAp #854-781 Quisque St.Or -73PetraMoon453-6609 Curabitur StreetCa -74ReeseEstradaAp #382-3313 Malesuada St.Or -75GageHiggins7443 Eu StreetCa -76ZacheryCamachoAp #795-4143 Quam. St.Or -77KellyGarnerP.O. Box 895, 2843 Cras Rd.Ca -78HanaeCarr9440 Amet St.Or -79AnnAlston884-7948 Dictum RoadCa -80ChancellorCobbP.O. Box 889, 5978 Ac AvenueOr -81DorothyHarrell6974 Tristique AveCa -82VaughanLeon1610 Luctus Av.Or -83WynneJimenez321-9171 Felis. AvenueCa -84WillaMendoza489-182 Sed Av.Or -85CamdenGoodwin4579 Ante St.Ca -86IfeomaFrenchP.O. Box 160, 8769 Integer RoadOr -87RamonaStrong1666 Ridiculus AvenueCa -88BrettRamosAp #579-9879 Et, RoadOr -89UllaGray595-7066 Malesuada RoadCa -90KevynMccallP.O. Box 968, 1420 Aenean AvenueOr -91GenevieveWilkins908 Turpis. StreetCa -92ThaneOneil6766 Lectus St.Or -93MarikoClineP.O. Box 329, 5375 Ac St.Ca -94LaelMclean500-7010 Sit St.Or -95WinifredHopperAp #140-8982 Velit AvenueCa -96RafaelEnglandP.O. Box 405, 7857 Eget Av.Or -97DanaCarter814-601 Purus. Av.Ca -98JulietBattleAp #535-1965 Cursus St.Or -99WynterVincent626-8492 Mollis AvenueCa -100WangMitchell4023 Lacinia. AveOr +1CelesteBrowning959-3763 Nec, Av.100.002011-03-12 15:20:00Ca +2RisaYangP.O. Box 292, 8229 Porttitor Road200.002011-03-12 15:20:00Or +3VenusSuttonAp #962-8021 Egestas Rd.300.002011-03-12 15:20:00Ca +4GretchenHarrisonP.O. Box 636, 8734 Magna Avenue400.002011-03-12 15:20:00Or +5LaniIrwinAp #441-5911 Iaculis, Ave500.002011-03-12 15:20:00Ca +6VeraGeorge409-1555 Vel, Ave600.002011-03-12 15:20:00Or +7JessicaMalone286-9779 Aliquam Road700.002011-03-12 15:20:00Ca +8AnnChapmanAp #504-3915 Placerat Road800.002011-03-12 15:20:00Or +9NigelBartlettAp #185-385 Diam Street900.002011-03-12 15:20:00Ca +10AzaliaJennings5772 Diam St.100.002011-03-12 15:20:00Or +11PrestonCannonAp #527-8769 Nunc Avenue100.002011-03-12 15:20:00Ca +12AllistairVasquez2562 Odio. St.100.002011-03-12 15:20:00Or +13ReedHayes5190 Elit Street100.002011-03-12 15:20:00Ca +14ElaineBarronP.O. Box 840, 8860 Sodales Av.100.002011-03-12 15:20:00Or +15LydiaHoodP.O. Box 698, 5666 Semper Road100.002011-03-12 15:20:00Ca +16VanceMaxwell298-3313 Malesuada Road100.002011-03-12 15:20:00Or +17KeikoDeleonP.O. Box 732, 5921 Massa. Av.100.002011-03-12 15:20:00Ca +18DolanKaneAp #906-3606 Ut Rd.100.002011-03-12 15:20:00Or +19MerrittPerkinsP.O. Box 228, 7090 Egestas Street100.002011-03-12 15:20:00Ca +20CaseySalazar506-5065 Ut St.200.002011-03-12 15:20:00Or +21SamsonNoel1370 Ultrices, Road200.002012-03-12 15:20:00Ca +22ByronWalkerP.O. Box 386, 8324 Tellus Ave200.002012-03-12 15:20:00Or +23PiperSingletonAp #500-3561 Primis St.200.002012-03-12 15:20:00Ca +24RiaMckinney3080 Dui Rd.200.002012-03-12 15:20:00Or +25RahimStanley559-9016 Nascetur Street200.002012-03-12 15:20:00Ca +26ChloeSteeleP.O. Box 766, 1628 Elit Street200.002012-03-12 15:20:00Or +27PalomaWardAp #390-3042 Ipsum Rd.200.002012-03-12 15:20:00Ca +28RoaryShermanAp #409-6549 Metus St.200.002012-03-12 15:20:00Or +29CalvinBuckner6378 Diam Avenue200.002012-03-12 15:20:00Ca +30CamilleGoodAp #113-8659 Suspendisse St.300.002012-03-12 15:20:00Or +31SteelAyala5518 Justo St.300.002012-03-12 15:20:00Ca +32JosiahGilbertAp #149-6651 At, Av.300.002012-03-12 15:20:00Or +33HamiltonCruz4620 Tellus. Ave300.002012-03-12 15:20:00Ca +34ScarletSantos586-1785 Velit. Av.300.002012-03-12 15:20:00Or +35LewisMcintyre629-6419 Ac Rd.300.002012-03-12 15:20:00Ca +36ArsenioMejiaP.O. Box 767, 8625 Justo Rd.300.002012-03-12 15:20:00Or +37VelmaHaley1377 At Rd.300.002012-03-12 15:20:00Ca +38TatumJennings829-7432 Posuere, Road300.002012-03-12 15:20:00Or +39BritanniEaton8811 Morbi Street300.002012-03-12 15:20:00Ca +40AileenJacobsonP.O. Box 469, 2266 Dui, Rd.400.002012-03-12 15:20:00Or +41KareemAyala2706 Ridiculus Street400.002013-03-12 15:20:00Ca +42MaiteRush7592 Neque Road400.002013-03-12 15:20:00Or +43SigneVelasquezAp #868-3039 Eget St.400.002013-03-12 15:20:00Ca +44ZoritaCamachoP.O. Box 651, 3340 Quis Av.400.002013-03-12 15:20:00Or +45GlennaCurtis953-7965 Enim Ave400.002013-03-12 15:20:00Ca +46QuinCortez4898 Ridiculus St.400.002013-03-12 15:20:00Or +47TalonDaltonP.O. Box 408, 7597 Integer Rd.400.002013-03-12 15:20:00Ca +48DarrylBlankenshipP.O. Box 771, 1471 Non Rd.400.002013-03-12 15:20:00Or +49VernonReyesP.O. Box 971, 7009 Vulputate Street400.002013-03-12 15:20:00Ca +50TallulahHeathP.O. Box 865, 3697 Dis Ave500.002013-03-12 15:20:00Or +51CiaranOlson2721 Et St.500.002013-03-12 15:20:00Ca +52OrlandoWittP.O. Box 717, 1102 Nulla. Rd.500.002013-03-12 15:20:00Or +53QuinnRiceAp #647-6627 Tristique Avenue500.002013-03-12 15:20:00Ca +54WyattPickettAp #128-3130 Vel, Rd.500.002013-03-12 15:20:00Or +55EmeraldCopeland857-5119 Turpis Rd.500.002013-03-12 15:20:00Ca +56JonasQuinnAp #441-7183 Ligula. Street500.002013-03-12 15:20:00Or +57WillaBerg6672 Velit Ave500.002013-03-12 15:20:00Ca +58MalikLee998-9208 In Street500.002013-03-12 15:20:00Or +59CallieMedina1620 Dui. Rd.500.002013-03-12 15:20:00Ca +60LukeMasonP.O. Box 143, 2070 Augue Rd.600.002013-03-12 15:20:00Or +61ShafiraEstrada8824 Ante Street600.002014-03-12 15:20:00Ca +62ElizabethRutledge315-6510 Sit St.600.002014-03-12 15:20:00Or +63PandoraLevine357-3596 Nibh. Ave600.002014-03-12 15:20:00Ca +64HilelPrince845-1229 Sociosqu Rd.600.002014-03-12 15:20:00Or +65RinahTorresAp #492-9328 At St.600.002014-03-12 15:20:00Ca +66YaelHobbsP.O. Box 477, 3896 In Street600.002014-03-12 15:20:00Or +67NevadaNashP.O. Box 251, 1914 Tincidunt Road600.002014-03-12 15:20:00Ca +68MarnyHuffP.O. Box 818, 6086 Ultricies St.600.002014-03-12 15:20:00Or +69KimberleyMilesAp #893-3685 In Road600.002014-03-12 15:20:00Ca +70DuncanFullerAp #197-5216 Iaculis Street700.002014-03-12 15:20:00Or +71YardleyLeblancP.O. Box 938, 1278 Sit Ave700.002014-03-12 15:20:00Ca +72HamishBrewerAp #854-781 Quisque St.700.002014-03-12 15:20:00Or +73PetraMoon453-6609 Curabitur Street700.002014-03-12 15:20:00Ca +74ReeseEstradaAp #382-3313 Malesuada St.700.002014-03-12 15:20:00Or +75GageHiggins7443 Eu Street700.002014-03-12 15:20:00Ca +76ZacheryCamachoAp #795-4143 Quam. St.700.002014-03-12 15:20:00Or +77KellyGarnerP.O. Box 895, 2843 Cras Rd.700.002014-03-12 15:20:00Ca +78HanaeCarr9440 Amet St.700.002014-03-12 15:20:00Or +79AnnAlston884-7948 Dictum Road700.002014-03-12 15:20:00Ca +80ChancellorCobbP.O. Box 889, 5978 Ac Avenue800.002014-03-12 15:20:00Or +81DorothyHarrell6974 Tristique Ave800.002010-03-12 15:20:00Ca +82VaughanLeon1610 Luctus Av.800.002010-03-12 15:20:00Or +83WynneJimenez321-9171 Felis. Avenue800.002010-03-12 15:20:00Ca +84WillaMendoza489-182 Sed Av.800.002010-03-12 15:20:00Or +85CamdenGoodwin4579 Ante St.800.002010-03-12 15:20:00Ca +86IfeomaFrenchP.O. Box 160, 8769 Integer Road800.002010-03-12 15:20:00Or +87RamonaStrong1666 Ridiculus Avenue800.002010-03-12 15:20:00Ca +88BrettRamosAp #579-9879 Et, Road800.002010-03-12 15:20:00Or +89UllaGray595-7066 Malesuada Road800.002010-03-12 15:20:00Ca +90KevynMccallP.O. Box 968, 1420 Aenean Avenue900.002010-03-12 15:20:00Or +91GenevieveWilkins908 Turpis. Street900.002010-03-12 15:20:00Ca +92ThaneOneil6766 Lectus St.900.002010-03-12 15:20:00Or +93MarikoClineP.O. Box 329, 5375 Ac St.900.002010-03-12 15:20:00Ca +94LaelMclean500-7010 Sit St.900.002010-03-12 15:20:00Or +95WinifredHopperAp #140-8982 Velit Avenue900.002010-03-12 15:20:00Ca +96RafaelEnglandP.O. Box 405, 7857 Eget Av.900.002010-03-12 15:20:00Or +97DanaCarter814-601 Purus. Av.900.002010-03-12 15:20:00Ca +98JulietBattleAp #535-1965 Cursus St.900.002010-03-12 15:20:00Or +99WynterVincent626-8492 Mollis Avenue900.002010-03-12 15:20:00Ca +100WangMitchell4023 Lacinia. Ave100.002010-03-12 15:20:00Or diff --git a/sql/hive/src/test/resources/data/files/orc_split_elim.orc b/sql/hive/src/test/resources/data/files/orc_split_elim.orc new file mode 100644 index 0000000000000000000000000000000000000000..cd145d343104983b4b09603c251ee749e5f82cc7 GIT binary patch literal 246402 zcmeI)Piz%Qeh2XCKhJI#W2WfAlWOyiO-X|e%1A= zyIz00-{)XU?*~8pejIgGbMIU{8Osp5Qhsl|IQQ^D|N7?R6JdBQtu6E;3)|1u;zF$QSl8u=tMjqA*j1NS zR>yr4kHxy5K74{{sKd_QZgKQxk|$5|;X8Z%L5FOUJh!Ood`4E=YhN8-0 zcn~}fz7Kr@o&(Q;=fL-X103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3Gc)uJ-`!`%G0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^yu$>hp$t2F;qYw| z!Zhr?c=ROZx%Z#Hl{z^%&k@$&4k6AoGi^w-LoL$jp_b{)P^)xqsC8N#>L|_UTC_#F zI@B^<$hB6q{(m8dJ^{~x=fHE|d%yt>aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<701MjZ`7xn?rNN`{L1)nCISq{#NSb;5;RY)6BFX%?`Clr-xdmGefP?xuMo+aj2s-pKH+;>FQ9+bRpNWEmNIqm1`55{C<_D zkIOildv&_kpJJ3d{nVlzr5nTkD4p-hCgiQ`>i*DIdwC>|x~jLMo*3;8T70<>ZbW8cY@#5V6EnZqXanM;9I`^Kf z#f4bqe%IxRtMjqA*j1NTE|2>r9*cE9ZTJM!a4p}N-QwuYBu}2^!*}-jgAUmyd2UhF z`4WZ;=$G~P`}g8BR$DBT$LN9lZ5 zHX(0iSNDg$+RGzx)KyJB@Kh{A=swKdE;e5m?QW62`s#XI9EMNj;pH|~pWXbt8{OXA z*ji6dCsjNfi_k6Q_r{BJzkb@kUVeOH|7I_(Ej)bi;KBE{pRL7(SmnX4%M(}UV{x&o zuB==c_f0$&>wf<55vJkmBHx?cKcJc9$nVy z!*k$!zyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N@tF?!b-x0JIX2fCMBU0SQPz0uqpb1SB8<2}nQ!5|F?ogAF!2RhW_o7m*{t2BLF#@XDf)4l!_qulAI7VRkA81_f$d{;IhZ)I2ahrZg&Bk%Cu zIJE>MAOQ(TKmrnwfCMBU0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^NI(J-kbndvAOQ(T zKmrnwfCMBU0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^NI(J-kbndvAc1$4Kpb^dlYevN zWX!*!(jCv;Zmx;#ZjtWXy&e~b(NlSJxsBE4`mJtsdvjxJJw2UN@oX$Yx0K%-FU~!D z(7(R<_{9F_Tv}V`M;5l9t;K~{<-xAY6IbVBaj~mDTKQ<)H}P1k`}xBsn1=pGZFY;J zHkm3)o8-AgRp(0>F5uwv_RISzcs~X2Iq*H;00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aNvD+;K%y`XeA&42}nQ!5|Drd zBp?9^NI(J-kbndvAc1$7z%-O$XD=MSO+uK4|9J7}Nz8NaKYuHAa&Vp_tiK&XoM~p- zkYD*B3v^dmJn$NXpi*$9UWx9}S*_Nr!waT@LO@6;h)5m3;&AmF^ z>rXMtoqlT3j?#@`f0WL5WfSsNc6EQ~tGzrDKaW+D4?LcmZmy}@-6Flbb3LBQ`^#;t zesuF@H@dyKv9+F_PO7++hsTR^4h&JSR?dI4J;WLm}?E1m@5sMm=YH- zGchtTu|(`Q<22x9f#b+b1XMYS;eE$kI*1xT_?4P^!=i2W- zyJjwX`0LZ`EpGo)7W*$232|(3Zb(~yXS2qJB_2{OndQnS9>`7C+Rb<*ihs?Glec*? zd{;DWY~Z+j`K5>9`Mpt27qumo4?RA+W4mU-?xp=3{(KI6Jb8M^F4e71SJt@{ueqnY zCc^K>uCq5P)8GGpw$W?nD%RC`6Am?n@jqO()9~5bo-?mIm@B^QjFE1(3uc;-AotQF z*nLmO%oQI6Ec3P=o02kP^Xb!}$9UJ2Fs$kce03oA@HZ#sDg1%A4OQc9q!PVU9Qs-c z(qx&K85tNCD;mff$O2G2LJL^K^Km?{LU?4Iw z2>&Vxlqi|Ipv+tTNYSaf{TVS?&61f~-_AZ_`)*r#ab~g?kD>aJ7T4&HR^P2mTfaFx ztJk@c7^?dwYxkOE-*>NgXJ8vM-Ni&~zr}z5Hn;wMd7sWXNoJ*|$~GKZ=^OpIdDFtr zO$X{MR?M2hI6bP~&x=>pA+vAqsbfJt_p0-Lhlw9L8EnSGOvrrqy3 zuGOiU*<9}OezjzQn~R*M#z|54jGYR*S6W|u-j(%s(gE?cSE{VHzp@ElcOWOn@!THU i+jZwF3Yv86jpP{Duj5kx`QB$`i> -Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389733869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2023038695216118221/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389733869}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:marmbrus, createTime:1413871682, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871682}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 index 2860d5c6b4cfb..1bfdd079010aa 100644 --- a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 +++ b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 @@ -1,5 +1,5 @@ -key int None +key int _bucketname string _offsets array -Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389733869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2023038695216118221/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389733869}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:marmbrus, createTime:1413871682, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871682}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d index ecafeaea5f61a..85c1918f46567 100644 --- a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d +++ b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d @@ -1,2 +1,2 @@ src -srcpart \ No newline at end of file +srcpart diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 b/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 index c80ef36c96ad4..136628776692b 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 @@ -1 +1 @@ -754 -7678496319 \ No newline at end of file +754 -7678496319 diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 index 618c1d01b726d..d32a641c0050e 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 @@ -1,12 +1,12 @@ -key int from deserializer -value string from deserializer -ds string None -ts string None +key int +value string +ds string +ts string # Partition Information # col_name data_type comment -ds string None -ts string None +ds string +ts string -Detailed Partition Information Partition(values:[2012-01-03, 2012-01-03+14:46:31], dbName:default, tableName:src_rc_merge_test_part, createTime:1389728902, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/src_rc_merge_test_part/ds=2012-01-03/ts=2012-01-03+14%3A46%3A31, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728902}) \ No newline at end of file +Detailed Partition Information Partition(values:[2012-01-03, 2012-01-03+14:46:31], dbName:default, tableName:src_rc_merge_test_part, createTime:1413930366, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6310217467672453297/src_rc_merge_test_part/ds=2012-01-03/ts=2012-01-03+14%3A46%3A31, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413930366}) diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f b/sql/hive/src/test/resources/golden/alter_merge_2-3-bc9bb363f9a2026cfc70a31bb4551352 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f rename to sql/hive/src/test/resources/golden/alter_merge_2-3-bc9bb363f9a2026cfc70a31bb4551352 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a b/sql/hive/src/test/resources/golden/alter_merge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a rename to sql/hive/src/test/resources/golden/alter_merge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 b/sql/hive/src/test/resources/golden/alter_merge_2-5-6319bf26f3739260b1a77e2ea89ef147 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 rename to sql/hive/src/test/resources/golden/alter_merge_2-5-6319bf26f3739260b1a77e2ea89ef147 diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 b/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 index 3f10ffe7a4c47..60d3b2f4a4cd5 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 @@ -1 +1 @@ -15 \ No newline at end of file +15 diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 b/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 index c80ef36c96ad4..136628776692b 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 @@ -1 +1 @@ -754 -7678496319 \ No newline at end of file +754 -7678496319 diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 b/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 index 3f10ffe7a4c47..60d3b2f4a4cd5 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 @@ -1 +1 @@ -15 \ No newline at end of file +15 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 b/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 deleted file mode 100644 index 410b14d2ce6f9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 +++ /dev/null @@ -1 +0,0 @@ -25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 b/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 deleted file mode 100644 index 410b14d2ce6f9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 +++ /dev/null @@ -1 +0,0 @@ -25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e b/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e deleted file mode 100644 index b28ab5ccf8a1b..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e +++ /dev/null @@ -1,75 +0,0 @@ -238 val_238 10 3.0 -NULL 10 3.0 -311 val_311 10 3.0 -NULL val_27 10 3.0 -NULL val_165 10 3.0 -NULL val_409 10 3.0 -255 val_255 10 3.0 -278 val_278 10 3.0 -98 val_98 10 3.0 -NULL val_484 10 3.0 -NULL val_265 10 3.0 -NULL val_193 10 3.0 -401 val_401 10 3.0 -150 val_150 10 3.0 -273 val_273 10 3.0 -224 10 3.0 -369 10 3.0 -66 val_66 10 3.0 -128 10 3.0 -213 val_213 10 3.0 -146 val_146 10 3.0 -406 val_406 10 3.0 -NULL 10 3.0 -NULL 10 3.0 -NULL 10 3.0 -238 val_238 100x 3.0 -NULL 100x 3.0 -311 val_311 100x 3.0 -NULL val_27 100x 3.0 -NULL val_165 100x 3.0 -NULL val_409 100x 3.0 -255 val_255 100x 3.0 -278 val_278 100x 3.0 -98 val_98 100x 3.0 -NULL val_484 100x 3.0 -NULL val_265 100x 3.0 -NULL val_193 100x 3.0 -401 val_401 100x 3.0 -150 val_150 100x 3.0 -273 val_273 100x 3.0 -224 100x 3.0 -369 100x 3.0 -66 val_66 100x 3.0 -128 100x 3.0 -213 val_213 100x 3.0 -146 val_146 100x 3.0 -406 val_406 100x 3.0 -NULL 100x 3.0 -NULL 100x 3.0 -NULL 100x 3.0 -238 val_238 100x 6:30pm -NULL 100x 6:30pm -311 val_311 100x 6:30pm -NULL val_27 100x 6:30pm -NULL val_165 100x 6:30pm -NULL val_409 100x 6:30pm -255 val_255 100x 6:30pm -278 val_278 100x 6:30pm -98 val_98 100x 6:30pm -NULL val_484 100x 6:30pm -NULL val_265 100x 6:30pm -NULL val_193 100x 6:30pm -401 val_401 100x 6:30pm -150 val_150 100x 6:30pm -273 val_273 100x 6:30pm -224 100x 6:30pm -369 100x 6:30pm -66 val_66 100x 6:30pm -128 100x 6:30pm -213 val_213 100x 6:30pm -146 val_146 100x 6:30pm -406 val_406 100x 6:30pm -NULL 100x 6:30pm -NULL 100x 6:30pm -NULL 100x 6:30pm \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c b/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c deleted file mode 100644 index c5b431b6cba29..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c +++ /dev/null @@ -1 +0,0 @@ -50 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 b/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 deleted file mode 100644 index a76c74dcec6ab..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 +++ /dev/null @@ -1 +0,0 @@ -75 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a b/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a deleted file mode 100644 index 316ca7f65ba20..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts string None - -# Partition Information -# col_name data_type comment - -dt string None -ts string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a b/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a deleted file mode 100644 index a7382fabfcb49..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts double None - -# Partition Information -# col_name data_type comment - -dt string None -ts double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 b/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 deleted file mode 100644 index a7382fabfcb49..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts double None - -# Partition Information -# col_name data_type comment - -dt string None -ts double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a b/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a deleted file mode 100644 index a7382fabfcb49..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts double None - -# Partition Information -# col_name data_type comment - -dt string None -ts double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 b/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 deleted file mode 100644 index a7382fabfcb49..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts double None - -# Partition Information -# col_name data_type comment - -dt string None -ts double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 b/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 deleted file mode 100644 index 410b14d2ce6f9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 +++ /dev/null @@ -1 +0,0 @@ -25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd b/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd deleted file mode 100644 index 410b14d2ce6f9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd +++ /dev/null @@ -1 +0,0 @@ -25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 b/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 index db182e444d31d..11487abed2b60 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,4 +1,4 @@ -key int None -value string None +key int +value string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871688}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 index 81e23f0bc1951..979969dcbfd3f 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 @@ -1,10 +1,10 @@ -key int from deserializer -value string from deserializer -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}) \ No newline at end of file +Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 index 6dc1f3ca2c187..7e14edcdead2e 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 @@ -1,10 +1,10 @@ -key int from deserializer -value string from deserializer -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}) \ No newline at end of file +Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 index 4b754043d63ab..77a764a814eb9 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,10 +1,10 @@ -key int None -value string None -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 index f44c28ee36760..c8606b1acad0c 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,10 +1,10 @@ -key int from deserializer -value string from deserializer -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388805892, transient_lastDdlTime=1388805892}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 index e739ad4992ec9..59922d3b7a086 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,10 +1,10 @@ -key int from deserializer -value string from deserializer -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388805892, transient_lastDdlTime=1388805892}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 index 092e5ed6e8a46..45ef755539479 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,4 +1,4 @@ -key int from deserializer -value string from deserializer +key int +value string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871688, transient_lastDdlTime=1413871688, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 index 197e67d09bf49..d6804307f3dc7 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,4 +1,4 @@ -key int from deserializer -value string from deserializer +key int +value string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871688, transient_lastDdlTime=1413871688, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 index 29b5b693b8589..77ba51afd2468 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 @@ -1,10 +1,10 @@ -key int None -value string None -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805891}) \ No newline at end of file +Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871689}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-e11f1476d2f30ab7a83f95013809f9e6 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-e11f1476d2f30ab7a83f95013809f9e6 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-29b4b67965f24c40bcc426043efb892d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-29b4b67965f24c40bcc426043efb892d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-2100fc7853625d8c7dad5c0ee70d4690 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-2100fc7853625d8c7dad5c0ee70d4690 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-c3fa911536c75e87febf2874e7889879 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-c3fa911536c75e87febf2874e7889879 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 index 02d72f4292749..c1b66aba46e84 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 +++ b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 @@ -9,4 +9,4 @@ 3 13 1996 12 7 17 1996 12 8 18 1996 12 -8 28 1996 12 \ No newline at end of file +8 28 1996 12 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d index bca47334cedaa..12ec4459dc167 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d +++ b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d @@ -3,4 +3,4 @@ 3 13 1995 09 7 17 1995 09 8 18 1995 09 -8 28 1995 09 \ No newline at end of file +8 28 1995 09 diff --git a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 +++ b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 index e881b2a4b74eb..605b17f1d8ceb 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 @@ -3,4 +3,4 @@ 3 new_part1: new_part2: 4 new_part1: new_part2: 5 new_part1: new_part2: -6 new_part1: new_part2: \ No newline at end of file +6 new_part1: new_part2: diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c index ecafeaea5f61a..85c1918f46567 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c @@ -1,2 +1,2 @@ src -srcpart \ No newline at end of file +srcpart diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f b/sql/hive/src/test/resources/golden/alter_rename_partition-19-d862c00e5a37ae841a6c8ec6c5d68e8c similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f rename to sql/hive/src/test/resources/golden/alter_rename_partition-19-d862c00e5a37ae841a6c8ec6c5d68e8c diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c index ecafeaea5f61a..85c1918f46567 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c @@ -1,2 +1,2 @@ src -srcpart \ No newline at end of file +srcpart diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 b/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 index 684f1da0fd0a6..6105de433830c 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 @@ -3,4 +3,4 @@ 3 old_part1: old_part2: 4 old_part1: old_part2: 5 old_part1: old_part2: -6 old_part1: old_part2: \ No newline at end of file +6 old_part1: old_part2: diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 b/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 index fc31ec62a1280..fb196b30f4478 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 @@ -1 +1 @@ -pcol1=new_part1%3A/pcol2=new_part2%3A \ No newline at end of file +pcol1=new_part1%3A/pcol2=new_part2%3A diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b b/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b index e881b2a4b74eb..605b17f1d8ceb 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b @@ -3,4 +3,4 @@ 3 new_part1: new_part2: 4 new_part1: new_part2: 5 new_part1: new_part2: -6 new_part1: new_part2: \ No newline at end of file +6 new_part1: new_part2: diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 b/sql/hive/src/test/resources/golden/alter_rename_partition-4-833254c8f7c5b1248e1a249560652627 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 rename to sql/hive/src/test/resources/golden/alter_rename_partition-4-833254c8f7c5b1248e1a249560652627 diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 index 684f1da0fd0a6..6105de433830c 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 @@ -3,4 +3,4 @@ 3 old_part1: old_part2: 4 old_part1: old_part2: 5 old_part1: old_part2: -6 old_part1: old_part2: \ No newline at end of file +6 old_part1: old_part2: diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 b/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 index fc31ec62a1280..fb196b30f4478 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 @@ -1 +1 @@ -pcol1=new_part1%3A/pcol2=new_part2%3A \ No newline at end of file +pcol1=new_part1%3A/pcol2=new_part2%3A diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 index ccd6518a50f7c..5a608a9064b84 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 @@ -1,5 +1,5 @@ -id int None -query string None -name string None +id int +query string +name string -Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871711}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 index d135e450e6e8e..1d6d48bbe1d1c 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711}) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 index d135e450e6e8e..1d6d48bbe1d1c 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711}) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 index c3fac88f91a36..f33269bd266be 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871712, transient_lastDdlTime=1413871712, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 index 6c8f91de7cded..41e8d696d33ea 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871712, transient_lastDdlTime=1413871712, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 index 37a5b2cc47bad..7caf0af50b071 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 @@ -1,5 +1,5 @@ -id int from deserializer -query string from deserializer -name string from deserializer +id int +query string +name string -Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:from deserializer), FieldSchema(name:query, type:string, comment:from deserializer), FieldSchema(name:name, type:string, comment:from deserializer)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871711, transient_lastDdlTime=1413871711, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 index 0348dd15fd4f5..a55479e69bbea 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 @@ -1,5 +1,5 @@ -id int from deserializer -query string from deserializer -name string from deserializer +id int +query string +name string -Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:from deserializer), FieldSchema(name:query, type:string, comment:from deserializer), FieldSchema(name:name, type:string, comment:from deserializer)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871711, transient_lastDdlTime=1413871711, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 index d135e450e6e8e..1d6d48bbe1d1c 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 b/sql/hive/src/test/resources/golden/alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 rename to sql/hive/src/test/resources/golden/alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 index 40818a7de46d0..600b377716894 100644 --- a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 +++ b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 @@ -1 +1 @@ -val_238 7 \ No newline at end of file +val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c index 827220bd4996f..ad69f390bc8db 100644 --- a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c +++ b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c @@ -1 +1 @@ -1 val_238 7 \ No newline at end of file +1 val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d b/sql/hive/src/test/resources/golden/alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d rename to sql/hive/src/test/resources/golden/alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d diff --git a/sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 rename to sql/hive/src/test/resources/golden/alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c deleted file mode 100644 index 827220bd4996f..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c +++ /dev/null @@ -1 +0,0 @@ -1 val_238 7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c b/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c deleted file mode 100644 index 8a8234a35f6bb..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c +++ /dev/null @@ -1 +0,0 @@ -2 238 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c new file mode 100644 index 0000000000000..ad69f390bc8db --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c @@ -0,0 +1 @@ +1 val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c b/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c new file mode 100644 index 0000000000000..1f8ddaec9003a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c @@ -0,0 +1 @@ +2 238 3 diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b index a99747531cef4..ecae907d92279 100644 --- a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b +++ b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b @@ -1,20 +1,20 @@ # col_name data_type comment -key int None -value string None -ds string None -hr string None +key int +value string +ds string +hr string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:47:52 PST 2014 +CreateTime: Mon Oct 20 23:10:25 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 Table Type: VIRTUAL_VIEW Table Parameters: - transient_lastDdlTime 1391813272 + transient_lastDdlTime 1413871825 # Storage Information SerDe Library: null diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b index 499c73127d890..10e3eca6c6791 100644 --- a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b +++ b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b @@ -1,17 +1,17 @@ # col_name data_type comment -value string None +value string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:47:52 PST 2014 +CreateTime: Mon Oct 20 23:10:25 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 Table Type: VIRTUAL_VIEW Table Parameters: - transient_lastDdlTime 1391813272 + transient_lastDdlTime 1413871825 # Storage Information SerDe Library: null diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b index a5fba77abdf07..c3661a1f79b57 100644 --- a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b +++ b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b @@ -1,18 +1,18 @@ # col_name data_type comment -key int None -value string None +key int +value string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:47:52 PST 2014 +CreateTime: Mon Oct 20 23:10:25 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 Table Type: VIRTUAL_VIEW Table Parameters: - transient_lastDdlTime 1391813272 + transient_lastDdlTime 1413871825 # Storage Information SerDe Library: null diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb b/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb deleted file mode 100644 index ee76e02af3aba..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb +++ /dev/null @@ -1,5 +0,0 @@ -foo int None -bar string None -ds string None - -Detailed Table Information Table(tableName:view1, dbName:default, owner:tnachen, createTime:1392426511, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:foo, type:int, comment:null), FieldSchema(name:bar, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426511}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 b/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 deleted file mode 100644 index 8603577477bfc..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 +++ /dev/null @@ -1,5 +0,0 @@ -foo int None -bar string None -ds string None - -Detailed Table Information Table(tableName:view2, dbName:default, owner:tnachen, createTime:1392426511, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:foo, type:int, comment:null), FieldSchema(name:bar, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426511, transient_lastDdlTime=1392426511}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) diff --git a/sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/ambiguous_col-0-e91e3e5a22029b9b979ccbbef97add66 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c rename to sql/hive/src/test/resources/golden/ambiguous_col-0-e91e3e5a22029b9b979ccbbef97add66 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d b/sql/hive/src/test/resources/golden/ambiguous_col-1-b4fe82679efdf6a15e9ecff53baf8d8d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d rename to sql/hive/src/test/resources/golden/ambiguous_col-1-b4fe82679efdf6a15e9ecff53baf8d8d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 b/sql/hive/src/test/resources/golden/ambiguous_col-2-dadfa3854356dead14b93c5a71a5d8ab similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 rename to sql/hive/src/test/resources/golden/ambiguous_col-2-dadfa3854356dead14b93c5a71a5d8ab diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 b/sql/hive/src/test/resources/golden/ambiguous_col-3-70509ccd2765d90b98666b6dff8afe1b similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 rename to sql/hive/src/test/resources/golden/ambiguous_col-3-70509ccd2765d90b98666b6dff8afe1b diff --git a/sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/annotate_stats_join-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to sql/hive/src/test/resources/golden/annotate_stats_join-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c b/sql/hive/src/test/resources/golden/annotate_stats_join-1-84e7846d50fc15e836c83911ce039871 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c rename to sql/hive/src/test/resources/golden/annotate_stats_join-1-84e7846d50fc15e836c83911ce039871 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 b/sql/hive/src/test/resources/golden/annotate_stats_join-10-a59dc1c01d48c82d46a5beab759f618d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 rename to sql/hive/src/test/resources/golden/annotate_stats_join-10-a59dc1c01d48c82d46a5beab759f618d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 b/sql/hive/src/test/resources/golden/annotate_stats_join-11-3e0ade2476221f6396381f55a3b82e60 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 rename to sql/hive/src/test/resources/golden/annotate_stats_join-11-3e0ade2476221f6396381f55a3b82e60 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 b/sql/hive/src/test/resources/golden/annotate_stats_join-12-cef44682d339a67ba765f854da21f976 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 rename to sql/hive/src/test/resources/golden/annotate_stats_join-12-cef44682d339a67ba765f854da21f976 diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c b/sql/hive/src/test/resources/golden/annotate_stats_join-13-6292c27f0c4eaeab2a5e6c317e3afa2e similarity index 100% rename from sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c rename to sql/hive/src/test/resources/golden/annotate_stats_join-13-6292c27f0c4eaeab2a5e6c317e3afa2e diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f b/sql/hive/src/test/resources/golden/annotate_stats_join-14-4f3042b9feebd00c540ddac40e7254d1 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f rename to sql/hive/src/test/resources/golden/annotate_stats_join-14-4f3042b9feebd00c540ddac40e7254d1 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 b/sql/hive/src/test/resources/golden/annotate_stats_join-15-a2f3b8a636e46e5df514df46c452855f similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 rename to sql/hive/src/test/resources/golden/annotate_stats_join-15-a2f3b8a636e46e5df514df46c452855f diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 b/sql/hive/src/test/resources/golden/annotate_stats_join-16-a75699a21ea36d962a1ba45bd5a12f26 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 rename to sql/hive/src/test/resources/golden/annotate_stats_join-16-a75699a21ea36d962a1ba45bd5a12f26 diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 b/sql/hive/src/test/resources/golden/annotate_stats_join-17-64380f94a431e3a1698aa2edd3d0a6b2 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 rename to sql/hive/src/test/resources/golden/annotate_stats_join-17-64380f94a431e3a1698aa2edd3d0a6b2 diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d b/sql/hive/src/test/resources/golden/annotate_stats_join-18-222d1fcce7420950436e00d3a1bba957 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d rename to sql/hive/src/test/resources/golden/annotate_stats_join-18-222d1fcce7420950436e00d3a1bba957 diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a b/sql/hive/src/test/resources/golden/annotate_stats_join-19-dea5f8f5c703583c7a3bdcb62cd3d589 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a rename to sql/hive/src/test/resources/golden/annotate_stats_join-19-dea5f8f5c703583c7a3bdcb62cd3d589 diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d b/sql/hive/src/test/resources/golden/annotate_stats_join-2-40548ec2313af8dbdcbb8ad0477d8600 similarity index 100% rename from sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d rename to sql/hive/src/test/resources/golden/annotate_stats_join-2-40548ec2313af8dbdcbb8ad0477d8600 diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab b/sql/hive/src/test/resources/golden/annotate_stats_join-20-2d5e186b97166e3a625a169d0d73f9c8 similarity index 100% rename from sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab rename to sql/hive/src/test/resources/golden/annotate_stats_join-20-2d5e186b97166e3a625a169d0d73f9c8 diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b b/sql/hive/src/test/resources/golden/annotate_stats_join-21-ed73d4b01424287148347ccf721b37e2 similarity index 100% rename from sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b rename to sql/hive/src/test/resources/golden/annotate_stats_join-21-ed73d4b01424287148347ccf721b37e2 diff --git a/sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_join-22-2cf93da6bb0efdafeaa989c9df236701 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 rename to sql/hive/src/test/resources/golden/annotate_stats_join-22-2cf93da6bb0efdafeaa989c9df236701 diff --git a/sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_join-23-fa90806f6137300311df032e28df3d4c similarity index 100% rename from sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 rename to sql/hive/src/test/resources/golden/annotate_stats_join-23-fa90806f6137300311df032e28df3d4c diff --git a/sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_join-24-4a6976344eeae35e059285ed78f9feb3 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0 rename to sql/hive/src/test/resources/golden/annotate_stats_join-24-4a6976344eeae35e059285ed78f9feb3 diff --git a/sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a b/sql/hive/src/test/resources/golden/annotate_stats_join-3-26f82fb6734d5bc6f7159b06c0949178 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a rename to sql/hive/src/test/resources/golden/annotate_stats_join-3-26f82fb6734d5bc6f7159b06c0949178 diff --git a/sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f b/sql/hive/src/test/resources/golden/annotate_stats_join-4-a598c93d86a646cfa6ea86da54bce9b8 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f rename to sql/hive/src/test/resources/golden/annotate_stats_join-4-a598c93d86a646cfa6ea86da54bce9b8 diff --git a/sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8 b/sql/hive/src/test/resources/golden/annotate_stats_join-5-d12ba848d20d1def95379c490068f050 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8 rename to sql/hive/src/test/resources/golden/annotate_stats_join-5-d12ba848d20d1def95379c490068f050 diff --git a/sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_join-6-96a4806e61c5d882affac57e115f285f similarity index 100% rename from sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 rename to sql/hive/src/test/resources/golden/annotate_stats_join-6-96a4806e61c5d882affac57e115f285f diff --git a/sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_join-7-38be41d6328f57350a78fb2097d1ebd2 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0 rename to sql/hive/src/test/resources/golden/annotate_stats_join-7-38be41d6328f57350a78fb2097d1ebd2 diff --git a/sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f b/sql/hive/src/test/resources/golden/annotate_stats_join-8-eb11e867a36e2114ef0f0f001e01984c similarity index 100% rename from sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f rename to sql/hive/src/test/resources/golden/annotate_stats_join-8-eb11e867a36e2114ef0f0f001e01984c diff --git a/sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a b/sql/hive/src/test/resources/golden/annotate_stats_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd similarity index 100% rename from sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a rename to sql/hive/src/test/resources/golden/annotate_stats_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd diff --git a/sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/annotate_stats_limit-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 b/sql/hive/src/test/resources/golden/annotate_stats_limit-1-26f82fb6734d5bc6f7159b06c0949178 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-1-26f82fb6734d5bc6f7159b06c0949178 diff --git a/sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 b/sql/hive/src/test/resources/golden/annotate_stats_limit-10-a89c94fd0de0cfc96725fea890987cc0 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-10-a89c94fd0de0cfc96725fea890987cc0 diff --git a/sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_limit-2-eb11e867a36e2114ef0f0f001e01984c similarity index 100% rename from sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-2-eb11e867a36e2114ef0f0f001e01984c diff --git a/sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd similarity index 100% rename from sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd diff --git a/sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_limit-4-4252006172a476fbc591eebee49bffa3 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-4-4252006172a476fbc591eebee49bffa3 diff --git a/sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_limit-5-a2f3b8a636e46e5df514df46c452855f similarity index 100% rename from sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-5-a2f3b8a636e46e5df514df46c452855f diff --git a/sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_limit-6-f1fd6c403a9787947877f48c772afe96 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-6-f1fd6c403a9787947877f48c772afe96 diff --git a/sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd b/sql/hive/src/test/resources/golden/annotate_stats_limit-7-69b422026fce4e2d2cde9a478d6aaa40 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd rename to sql/hive/src/test/resources/golden/annotate_stats_limit-7-69b422026fce4e2d2cde9a478d6aaa40 diff --git a/sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_limit-8-72f5e55c1e244084eea255c32a6f58eb similarity index 100% rename from sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca rename to sql/hive/src/test/resources/golden/annotate_stats_limit-8-72f5e55c1e244084eea255c32a6f58eb diff --git a/sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_limit-9-9da67c62d5e3145d450ad7953b33361f similarity index 100% rename from sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c rename to sql/hive/src/test/resources/golden/annotate_stats_limit-9-9da67c62d5e3145d450ad7953b33361f diff --git a/sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/annotate_stats_part-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf rename to sql/hive/src/test/resources/golden/annotate_stats_part-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/annotate_stats_part-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to sql/hive/src/test/resources/golden/annotate_stats_part-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_part-10-5ba0295bfe42deb678e59b3a330b14ff similarity index 100% rename from sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf rename to sql/hive/src/test/resources/golden/annotate_stats_part-10-5ba0295bfe42deb678e59b3a330b14ff diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_part-11-dbdbe2e04c5dad6c8af78b6386b329 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 rename to sql/hive/src/test/resources/golden/annotate_stats_part-11-dbdbe2e04c5dad6c8af78b6386b329 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_part-12-2856fb4f4af8932804cb238429d9de6f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 rename to sql/hive/src/test/resources/golden/annotate_stats_part-12-2856fb4f4af8932804cb238429d9de6f diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_part-13-4fa8b0f9fb0202ac0e82fb87538d6445 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 rename to sql/hive/src/test/resources/golden/annotate_stats_part-13-4fa8b0f9fb0202ac0e82fb87538d6445 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a b/sql/hive/src/test/resources/golden/annotate_stats_part-14-62c557bfb7bf70694a32ebceea329ebd similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a rename to sql/hive/src/test/resources/golden/annotate_stats_part-14-62c557bfb7bf70694a32ebceea329ebd diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f b/sql/hive/src/test/resources/golden/annotate_stats_part-15-f796cd035603726a5c4ce3e71194822b similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f rename to sql/hive/src/test/resources/golden/annotate_stats_part-15-f796cd035603726a5c4ce3e71194822b diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 b/sql/hive/src/test/resources/golden/annotate_stats_part-16-45eb5544124364714549f199f9c2b2ac similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 rename to sql/hive/src/test/resources/golden/annotate_stats_part-16-45eb5544124364714549f199f9c2b2ac diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_part-17-ad61ebd912b6bef3778c4ff38c0be5ab similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 rename to sql/hive/src/test/resources/golden/annotate_stats_part-17-ad61ebd912b6bef3778c4ff38c0be5ab diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_part-18-b9f2dff7c7b57412cea44433ea810fa7 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 rename to sql/hive/src/test/resources/golden/annotate_stats_part-18-b9f2dff7c7b57412cea44433ea810fa7 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f b/sql/hive/src/test/resources/golden/annotate_stats_part-19-10cab43c2966718bb39e2f22365cd6c1 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f rename to sql/hive/src/test/resources/golden/annotate_stats_part-19-10cab43c2966718bb39e2f22365cd6c1 diff --git a/sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/annotate_stats_part-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 rename to sql/hive/src/test/resources/golden/annotate_stats_part-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a b/sql/hive/src/test/resources/golden/annotate_stats_part-20-51ec5046b50543899ed54c9fc2b885af similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a rename to sql/hive/src/test/resources/golden/annotate_stats_part-20-51ec5046b50543899ed54c9fc2b885af diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 b/sql/hive/src/test/resources/golden/annotate_stats_part-21-d69cb7b7770b51bc4b99d5d0f74d4662 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 rename to sql/hive/src/test/resources/golden/annotate_stats_part-21-d69cb7b7770b51bc4b99d5d0f74d4662 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 b/sql/hive/src/test/resources/golden/annotate_stats_part-22-22f430881fbfe92736988c2a4184fe0c similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 rename to sql/hive/src/test/resources/golden/annotate_stats_part-22-22f430881fbfe92736988c2a4184fe0c diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_part-23-2719ee3f0b8b07f1e5c80f8329b9f87f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_part-23-2719ee3f0b8b07f1e5c80f8329b9f87f diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_part-24-1f7bdd816b83f45a6d53d08b6866109f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 rename to sql/hive/src/test/resources/golden/annotate_stats_part-24-1f7bdd816b83f45a6d53d08b6866109f diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_part-25-d1599e385e8bad6cd62d564a6fd0536f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_part-25-d1599e385e8bad6cd62d564a6fd0536f diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_part-26-ec26bcb2587f4638b364ba494ac373e0 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 rename to sql/hive/src/test/resources/golden/annotate_stats_part-26-ec26bcb2587f4638b364ba494ac373e0 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_part-27-7804e2e230a42a46d72546fa2381b423 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 rename to sql/hive/src/test/resources/golden/annotate_stats_part-27-7804e2e230a42a46d72546fa2381b423 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd b/sql/hive/src/test/resources/golden/annotate_stats_part-28-40f2a1f06d181ef93edf42a591cbf15e similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd rename to sql/hive/src/test/resources/golden/annotate_stats_part-28-40f2a1f06d181ef93edf42a591cbf15e diff --git a/sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/annotate_stats_part-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to sql/hive/src/test/resources/golden/annotate_stats_part-3-16367c381d4b189b3640c92511244bfe diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_part-4-397e834a25a20b68aed5e87d55ea6174 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca rename to sql/hive/src/test/resources/golden/annotate_stats_part-4-397e834a25a20b68aed5e87d55ea6174 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_part-5-cef44682d339a67ba765f854da21f976 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c rename to sql/hive/src/test/resources/golden/annotate_stats_part-5-cef44682d339a67ba765f854da21f976 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_part-6-f4263aca1612c05cb08242448191ad05 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf rename to sql/hive/src/test/resources/golden/annotate_stats_part-6-f4263aca1612c05cb08242448191ad05 diff --git a/sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_part-7-b222533ddda09918c4f5acc408bf1a02 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 rename to sql/hive/src/test/resources/golden/annotate_stats_part-7-b222533ddda09918c4f5acc408bf1a02 diff --git a/sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 b/sql/hive/src/test/resources/golden/annotate_stats_part-8-84e3979946de67779a9704a3adc2184f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 rename to sql/hive/src/test/resources/golden/annotate_stats_part-8-84e3979946de67779a9704a3adc2184f diff --git a/sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d b/sql/hive/src/test/resources/golden/annotate_stats_part-9-c6c38e93babafe56cd4f177a17d37b8 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d rename to sql/hive/src/test/resources/golden/annotate_stats_part-9-c6c38e93babafe56cd4f177a17d37b8 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/annotate_stats_table-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b rename to sql/hive/src/test/resources/golden/annotate_stats_table-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/annotate_stats_table-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f rename to sql/hive/src/test/resources/golden/annotate_stats_table-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_table-10-a7419af512a6c0b60ec51e556608cdad similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 rename to sql/hive/src/test/resources/golden/annotate_stats_table-10-a7419af512a6c0b60ec51e556608cdad diff --git a/sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_table-11-4be11140c2e21b3a172224d103844785 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_table-11-4be11140c2e21b3a172224d103844785 diff --git a/sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_table-12-84bc7cf67544461e9102820ed410dfb6 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 rename to sql/hive/src/test/resources/golden/annotate_stats_table-12-84bc7cf67544461e9102820ed410dfb6 diff --git a/sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_table-13-d2acfdf279aad035b31ed61e87bff6 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 rename to sql/hive/src/test/resources/golden/annotate_stats_table-13-d2acfdf279aad035b31ed61e87bff6 diff --git a/sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/annotate_stats_table-14-60d1c44b8412c4983265e320ecfb25f0 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a rename to sql/hive/src/test/resources/golden/annotate_stats_table-14-60d1c44b8412c4983265e320ecfb25f0 diff --git a/sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_table-15-7b185296b6a0c6ceff2df1f89fecc6b4 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca rename to sql/hive/src/test/resources/golden/annotate_stats_table-15-7b185296b6a0c6ceff2df1f89fecc6b4 diff --git a/sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c rename to sql/hive/src/test/resources/golden/annotate_stats_table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d diff --git a/sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_table-17-cd84d629d569a09d139f38dae01e677c similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf rename to sql/hive/src/test/resources/golden/annotate_stats_table-17-cd84d629d569a09d139f38dae01e677c diff --git a/sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf b/sql/hive/src/test/resources/golden/annotate_stats_table-18-7012af4092c18075f4f3a00b80f491ee similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf rename to sql/hive/src/test/resources/golden/annotate_stats_table-18-7012af4092c18075f4f3a00b80f491ee diff --git a/sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 b/sql/hive/src/test/resources/golden/annotate_stats_table-2-84e7846d50fc15e836c83911ce039871 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 rename to sql/hive/src/test/resources/golden/annotate_stats_table-2-84e7846d50fc15e836c83911ce039871 diff --git a/sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e b/sql/hive/src/test/resources/golden/annotate_stats_table-3-a598c93d86a646cfa6ea86da54bce9b8 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e rename to sql/hive/src/test/resources/golden/annotate_stats_table-3-a598c93d86a646cfa6ea86da54bce9b8 diff --git a/sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f b/sql/hive/src/test/resources/golden/annotate_stats_table-4-d12ba848d20d1def95379c490068f050 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f rename to sql/hive/src/test/resources/golden/annotate_stats_table-4-d12ba848d20d1def95379c490068f050 diff --git a/sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 b/sql/hive/src/test/resources/golden/annotate_stats_table-5-4a8ad142e57c5dce2623e92f5e5ad15a similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 rename to sql/hive/src/test/resources/golden/annotate_stats_table-5-4a8ad142e57c5dce2623e92f5e5ad15a diff --git a/sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 b/sql/hive/src/test/resources/golden/annotate_stats_table-6-a59dc1c01d48c82d46a5beab759f618d similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 rename to sql/hive/src/test/resources/golden/annotate_stats_table-6-a59dc1c01d48c82d46a5beab759f618d diff --git a/sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b b/sql/hive/src/test/resources/golden/annotate_stats_table-7-6292c27f0c4eaeab2a5e6c317e3afa2e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b rename to sql/hive/src/test/resources/golden/annotate_stats_table-7-6292c27f0c4eaeab2a5e6c317e3afa2e diff --git a/sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c b/sql/hive/src/test/resources/golden/annotate_stats_table-8-43cb040e2fe01904bc52e198fcae9b3d similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c rename to sql/hive/src/test/resources/golden/annotate_stats_table-8-43cb040e2fe01904bc52e198fcae9b3d diff --git a/sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 b/sql/hive/src/test/resources/golden/annotate_stats_table-9-eadfdc61d22bd22bbf5a69370908a82e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 rename to sql/hive/src/test/resources/golden/annotate_stats_table-9-eadfdc61d22bd22bbf5a69370908a82e diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/annotate_stats_union-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 rename to sql/hive/src/test/resources/golden/annotate_stats_union-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 b/sql/hive/src/test/resources/golden/annotate_stats_union-1-26f82fb6734d5bc6f7159b06c0949178 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 rename to sql/hive/src/test/resources/golden/annotate_stats_union-1-26f82fb6734d5bc6f7159b06c0949178 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/annotate_stats_union-10-3aa3eb68d092ea0157c5426a5f2ca3f9 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b rename to sql/hive/src/test/resources/golden/annotate_stats_union-10-3aa3eb68d092ea0157c5426a5f2ca3f9 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/annotate_stats_union-11-c1302094d7b4456550826535b529468b similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 rename to sql/hive/src/test/resources/golden/annotate_stats_union-11-c1302094d7b4456550826535b529468b diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/annotate_stats_union-12-3af760f960a2e0995d1bc4afef0c5aa0 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c rename to sql/hive/src/test/resources/golden/annotate_stats_union-12-3af760f960a2e0995d1bc4afef0c5aa0 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/annotate_stats_union-13-26f82fb6734d5bc6f7159b06c0949178 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 rename to sql/hive/src/test/resources/golden/annotate_stats_union-13-26f82fb6734d5bc6f7159b06c0949178 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 b/sql/hive/src/test/resources/golden/annotate_stats_union-14-eb11e867a36e2114ef0f0f001e01984c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 rename to sql/hive/src/test/resources/golden/annotate_stats_union-14-eb11e867a36e2114ef0f0f001e01984c diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 b/sql/hive/src/test/resources/golden/annotate_stats_union-15-e0b7cb2226234fbea8ae11ea7d9d84bd similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 rename to sql/hive/src/test/resources/golden/annotate_stats_union-15-e0b7cb2226234fbea8ae11ea7d9d84bd diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 b/sql/hive/src/test/resources/golden/annotate_stats_union-16-4252006172a476fbc591eebee49bffa3 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 rename to sql/hive/src/test/resources/golden/annotate_stats_union-16-4252006172a476fbc591eebee49bffa3 diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b b/sql/hive/src/test/resources/golden/annotate_stats_union-17-a2f3b8a636e46e5df514df46c452855f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b rename to sql/hive/src/test/resources/golden/annotate_stats_union-17-a2f3b8a636e46e5df514df46c452855f diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/annotate_stats_union-18-58d55f8c8b2489726232a00254499144 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/annotate_stats_union-18-58d55f8c8b2489726232a00254499144 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/annotate_stats_union-19-70c851c23a7ac89f0366b9c26085cc3e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 rename to sql/hive/src/test/resources/golden/annotate_stats_union-19-70c851c23a7ac89f0366b9c26085cc3e diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/annotate_stats_union-2-eb11e867a36e2114ef0f0f001e01984c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to sql/hive/src/test/resources/golden/annotate_stats_union-2-eb11e867a36e2114ef0f0f001e01984c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/annotate_stats_union-20-222d1fcce7420950436e00d3a1bba957 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/annotate_stats_union-20-222d1fcce7420950436e00d3a1bba957 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/annotate_stats_union-21-a5503df8f50e057415719bcfe4c69f13 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/annotate_stats_union-21-a5503df8f50e057415719bcfe4c69f13 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/annotate_stats_union-22-a23089e5a76dc35df04d9936d60c4565 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/annotate_stats_union-22-a23089e5a76dc35df04d9936d60c4565 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/annotate_stats_union-3-e0b7cb2226234fbea8ae11ea7d9d84bd similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/annotate_stats_union-3-e0b7cb2226234fbea8ae11ea7d9d84bd diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/annotate_stats_union-4-4252006172a476fbc591eebee49bffa3 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/annotate_stats_union-4-4252006172a476fbc591eebee49bffa3 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/annotate_stats_union-5-a2f3b8a636e46e5df514df46c452855f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/annotate_stats_union-5-a2f3b8a636e46e5df514df46c452855f diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/annotate_stats_union-6-222d1fcce7420950436e00d3a1bba957 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/annotate_stats_union-6-222d1fcce7420950436e00d3a1bba957 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/annotate_stats_union-7-aab1a7b7e1fe4d061a580126d67dfd0a similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/annotate_stats_union-7-aab1a7b7e1fe4d061a580126d67dfd0a diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/annotate_stats_union-8-172e0d089bd5bcbaf54775a618d826bb similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 rename to sql/hive/src/test/resources/golden/annotate_stats_union-8-172e0d089bd5bcbaf54775a618d826bb diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/annotate_stats_union-9-69b422026fce4e2d2cde9a478d6aaa40 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to sql/hive/src/test/resources/golden/annotate_stats_union-9-69b422026fce4e2d2cde9a478d6aaa40 diff --git a/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 deleted file mode 100644 index 21b3b13a81191..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 +++ /dev/null @@ -1 +0,0 @@ -0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 deleted file mode 100644 index 5e5f6ff96623f..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 +++ /dev/null @@ -1,9 +0,0 @@ -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 b/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 deleted file mode 100644 index 18a1a7925ff29..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 +++ /dev/null @@ -1 +0,0 @@ -48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a deleted file mode 100644 index 18a1a7925ff29..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a +++ /dev/null @@ -1 +0,0 @@ -48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 deleted file mode 100644 index 21b3b13a81191..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 +++ /dev/null @@ -1 +0,0 @@ -0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 deleted file mode 100644 index 5e5f6ff96623f..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 +++ /dev/null @@ -1,9 +0,0 @@ -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 deleted file mode 100644 index 18a1a7925ff29..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 +++ /dev/null @@ -1 +0,0 @@ -48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a deleted file mode 100644 index 18a1a7925ff29..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a +++ /dev/null @@ -1 +0,0 @@ -48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e b/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 deleted file mode 100644 index 21b3b13a81191..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 +++ /dev/null @@ -1 +0,0 @@ -0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 deleted file mode 100644 index 5e5f6ff96623f..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 +++ /dev/null @@ -1,9 +0,0 @@ -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba b/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba +++ b/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c b/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c index 308fc0924e670..8e9be7f8c620b 100644 --- a/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c +++ b/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c @@ -1 +1 @@ -34298511120 \ No newline at end of file +34298511120 diff --git a/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a b/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a +++ b/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e b/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e +++ b/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 b/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 +++ b/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 b/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 index 69dbf8c3143e9..faeebdd1eafdf 100644 --- a/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 +++ b/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 @@ -1 +1 @@ --101339664144 \ No newline at end of file +-101339664144 diff --git a/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 b/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 index eff107c7ce6bc..caf74830fbd05 100644 --- a/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 +++ b/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 @@ -1 +1 @@ --136852761207 \ No newline at end of file +-136852761207 diff --git a/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 b/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 index de6c015da2059..e971081015515 100644 --- a/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 +++ b/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 @@ -1 +1 @@ --97676500536 \ No newline at end of file +-97676500536 diff --git a/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e +++ b/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 +++ b/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 +++ b/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 b/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 index f1871a4957ddb..0f27a9bde401c 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 +++ b/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 @@ -1 +1 @@ -404554174174 \ No newline at end of file +404554174174 diff --git a/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 b/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 index 006e1f82c0a47..31d87ddbd1310 100644 --- a/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 +++ b/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 @@ -1 +1 @@ --793937029770 \ No newline at end of file +-793937029770 diff --git a/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 b/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c b/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c index 006e1f82c0a47..31d87ddbd1310 100644 --- a/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c +++ b/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c @@ -1 +1 @@ --793937029770 \ No newline at end of file +-793937029770 diff --git a/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f b/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f index 0c9b518e65ece..42af6f2e56afc 100644 --- a/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f +++ b/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f @@ -1 +1 @@ -2358131334 \ No newline at end of file +2358131334 diff --git a/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f index 795166629df40..069b64b649977 100644 --- a/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f +++ b/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f @@ -1 +1 @@ -407444119660 \ No newline at end of file +407444119660 diff --git a/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 b/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 index 1c958900f5013..b8f473bf53aa3 100644 --- a/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 +++ b/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 @@ -1 +1 @@ -33815990627 \ No newline at end of file +33815990627 diff --git a/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 b/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 index b1a6075f768c8..6b72a1789f57d 100644 --- a/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 +++ b/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 @@ -1 +1 @@ --24276731469 \ No newline at end of file +-24276731469 diff --git a/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 b/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 index b1a6075f768c8..6b72a1789f57d 100644 --- a/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 +++ b/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 @@ -1 +1 @@ --24276731469 \ No newline at end of file +-24276731469 diff --git a/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f b/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f index 9672e21fa0323..80c230cf4a09d 100644 --- a/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f +++ b/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f @@ -497,4 +497,4 @@ NULL NULL NULL NULL 496 val_496 NULL NULL NULL NULL 497 val_497 NULL NULL NULL NULL 498 val_498 NULL NULL NULL NULL 498 val_498 -NULL NULL NULL NULL 498 val_498 \ No newline at end of file +NULL NULL NULL NULL 498 val_498 diff --git a/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb b/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb index dba3bca53f72d..79162030a2043 100644 --- a/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb +++ b/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb @@ -1 +1 @@ -344337359100 \ No newline at end of file +344337359100 diff --git a/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 b/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 index 5707ed08e7e54..13e88f30fc08c 100644 --- a/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 +++ b/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 @@ -97,4 +97,4 @@ 9 val_9 5 val_5 9 val_9 5 val_5 9 val_9 8 val_8 -9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 b/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 +++ b/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa index 71094ee7360db..16b313fc58f23 100644 --- a/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -12,4 +12,4 @@ 311 3 369 3 401 5 -406 4 \ No newline at end of file +406 4 diff --git a/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e b/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e index dd35c6b71fc80..a16667d785bf8 100644 --- a/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e +++ b/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e @@ -1 +1 @@ -548 \ No newline at end of file +548 diff --git a/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c b/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c +++ b/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f +++ b/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a b/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a +++ b/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d b/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d +++ b/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 b/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 +++ b/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 b/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 +++ b/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 b/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 +++ b/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 b/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 +++ b/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 b/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 +++ b/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 b/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 +++ b/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 b/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 +++ b/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_join32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_join32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_join32-13-ee2dcaae78ae900ffce8d19fbadc3735 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_join32-13-ee2dcaae78ae900ffce8d19fbadc3735 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_join32-14-7927c2ce644d1ce1de251405c8563e99 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_join32-14-7927c2ce644d1ce1de251405c8563e99 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_join32-15-4cd3b51861720ac06c6deb818c83670 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/auto_join32-15-4cd3b51861720ac06c6deb818c83670 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_join32-20-693874ea8e06a8b155394ac27af2b1a7 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/auto_join32-20-693874ea8e06a8b155394ac27af2b1a7 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_join32-21-bf8a1bb0baaae9fbf1c3aa656f991f42 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_join32-21-bf8a1bb0baaae9fbf1c3aa656f991f42 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_join32-22-3d14d63e996851f51a98f987995d8da6 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_join32-22-3d14d63e996851f51a98f987995d8da6 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_join32-23-8b183ec2c164b3b530e802ffc880a5fa similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_join32-23-8b183ec2c164b3b530e802ffc880a5fa diff --git a/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef +++ b/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 index f7d1e92d77207..9e503cc0fbeb9 100644 --- a/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 +++ b/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 @@ -1 +1 @@ -5079148035 \ No newline at end of file +5079148035 diff --git a/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 index e859b7c4ada7b..1b298efae95b9 100644 --- a/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 +++ b/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 @@ -1 +1 @@ -9766083196 \ No newline at end of file +9766083196 diff --git a/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 index f15ab2fb14eaa..1e9e1b54b2074 100644 --- a/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 +++ b/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 @@ -1 +1 @@ -2607643291 \ No newline at end of file +2607643291 diff --git a/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 b/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 index 5f7f06c079d24..5ac39f668df1c 100644 --- a/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 +++ b/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 @@ -1 +1 @@ --2315698213 \ No newline at end of file +-2315698213 diff --git a/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f +++ b/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_join_filters-2-bee6095f42de6a16708c2f9addc1b9bd similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 rename to sql/hive/src/test/resources/golden/auto_join_filters-2-bee6095f42de6a16708c2f9addc1b9bd diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_join_filters-31-268d8fb3cb9b04eb269fe7ec40a24dfe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to sql/hive/src/test/resources/golden/auto_join_filters-31-268d8fb3cb9b04eb269fe7ec40a24dfe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a b/sql/hive/src/test/resources/golden/auto_join_filters-32-6dc6866a65c74d69538b776b41b06c16 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a rename to sql/hive/src/test/resources/golden/auto_join_filters-32-6dc6866a65c74d69538b776b41b06c16 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 b/sql/hive/src/test/resources/golden/auto_join_filters-33-e884480a0f7273d3e2f2de2ba46b855c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 rename to sql/hive/src/test/resources/golden/auto_join_filters-33-e884480a0f7273d3e2f2de2ba46b855c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 b/sql/hive/src/test/resources/golden/auto_join_filters-34-98fd86aea9cacaa82d43c7468109dd33 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 rename to sql/hive/src/test/resources/golden/auto_join_filters-34-98fd86aea9cacaa82d43c7468109dd33 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 b/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 index b201b9c4d9348..acf9ebdfa9579 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 @@ -1 +1 @@ -4542003 \ No newline at end of file +4542003 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 b/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 index d365cdf04366c..4380aa676ba67 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 @@ -1 +1 @@ -4542038 \ No newline at end of file +4542038 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c b/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c index bc7bcdca25bfb..359888de9ce3c 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c @@ -1 +1 @@ -4543491 \ No newline at end of file +4543491 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 b/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 index b201b9c4d9348..acf9ebdfa9579 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 @@ -1 +1 @@ -4542003 \ No newline at end of file +4542003 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 b/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 index feea6ee0a8e0d..3b196ba0b9f87 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 @@ -1 +1 @@ -3079923 \ No newline at end of file +3079923 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 b/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 index f713b04028bbd..dba80cf2f3b4b 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 @@ -1 +1 @@ -4509891 \ No newline at end of file +4509891 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af b/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af index a94eda6b2c374..7e29fae3a0aa6 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af @@ -1 +1 @@ -3113558 \ No newline at end of file +3113558 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 b/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 index feea6ee0a8e0d..3b196ba0b9f87 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 @@ -1 +1 @@ -3079923 \ No newline at end of file +3079923 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb b/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb index 88c5f95e0d838..a4231499b4e56 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb @@ -1 +1 @@ -4543526 \ No newline at end of file +4543526 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 b/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 index 88c5f95e0d838..a4231499b4e56 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 @@ -1 +1 @@ -4543526 \ No newline at end of file +4543526 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_join_nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_join_nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d b/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d index 88c5f95e0d838..a4231499b4e56 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d @@ -1 +1 @@ -4543526 \ No newline at end of file +4543526 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a b/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a index 88c5f95e0d838..a4231499b4e56 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a @@ -1 +1 @@ -4543526 \ No newline at end of file +4543526 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 b/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 index f2ec932ce57f4..2fa702a938a45 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 @@ -1 +1 @@ -3112070 \ No newline at end of file +3112070 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa b/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa index a94eda6b2c374..7e29fae3a0aa6 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa @@ -1 +1 @@ -3113558 \ No newline at end of file +3113558 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 b/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 index f2ec932ce57f4..2fa702a938a45 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 @@ -1 +1 @@ -3112070 \ No newline at end of file +3112070 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 b/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 index 4125efd2dd065..27994c451682a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 @@ -1 +1 @@ -13630578 \ No newline at end of file +13630578 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 b/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 index 4125efd2dd065..27994c451682a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 @@ -1 +1 @@ -13630578 \ No newline at end of file +13630578 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc b/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc index 4125efd2dd065..27994c451682a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc @@ -1 +1 @@ -13630578 \ No newline at end of file +13630578 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 b/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 index e877d44372ecb..476d8eeee571a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 @@ -1 +1 @@ -3078400 \ No newline at end of file +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 b/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 index 18be36a9bdb54..935eec97c5601 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 @@ -1 +1 @@ -4509856 \ No newline at end of file +4509856 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 b/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 index f2ec932ce57f4..2fa702a938a45 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 @@ -1 +1 @@ -3112070 \ No newline at end of file +3112070 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e b/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e index e877d44372ecb..476d8eeee571a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e @@ -1 +1 @@ -3078400 \ No newline at end of file +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_join_reordering_values-1-2bfb628930d072124636d21d82e3b462 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-1-2bfb628930d072124636d21d82e3b462 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_join_reordering_values-4-11af6838bb9e04152c2f9a7e2044abe0 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-4-11af6838bb9e04152c2f9a7e2044abe0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-7-99fcaa5203ed3debb52c9086028dc8c2 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-7-99fcaa5203ed3debb52c9086028dc8c2 diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed b/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed +++ b/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 index 62f9457511f87..1e8b314962144 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -1 +1 @@ -6 \ No newline at end of file +6 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 index 2eafac63a9a98..2a51623eae15c 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 @@ -3,4 +3,4 @@ 4 1 1 5 9 9 8 1 1 -9 1 1 \ No newline at end of file +9 1 1 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce index 2ebc6516c7df1..f6b91e0e1f8dd 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce @@ -1 +1 @@ -56 \ No newline at end of file +56 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e index 3d2e6576f591f..f892bae472dff 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e @@ -19,4 +19,4 @@ 5 val_5 5 val_5 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 index 34d56da297220..74ff4beddf949 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 @@ -19,4 +19,4 @@ 5 val_5 val_5 5 val_5 val_5 8 val_8 val_8 -9 val_9 val_9 \ No newline at end of file +9 val_9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e index 3d2e6576f591f..f892bae472dff 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e @@ -19,4 +19,4 @@ 5 val_5 5 val_5 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 index 4a9735f855f96..ec7496a567609 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 @@ -3,4 +3,4 @@ 4 1 5 9 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-5cba470fbd02e730781a3b63fd9aa3e2 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-5cba470fbd02e730781a3b63fd9aa3e2 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-337e909605c780d00ad8895686defa06 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-337e909605c780d00ad8895686defa06 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 index 86ee83a4a2686..425151f3a411f 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 @@ -1 +1 @@ -40 \ No newline at end of file +40 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e index 301160a93062d..45a4fb75db864 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e @@ -1 +1 @@ -8 \ No newline at end of file +8 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-5cba470fbd02e730781a3b63fd9aa3e2 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-5cba470fbd02e730781a3b63fd9aa3e2 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-337e909605c780d00ad8895686defa06 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-337e909605c780d00ad8895686defa06 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-5cba470fbd02e730781a3b63fd9aa3e2 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-5cba470fbd02e730781a3b63fd9aa3e2 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-337e909605c780d00ad8895686defa06 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-337e909605c780d00ad8895686defa06 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-97ba394ab6aad2547f399ebbf757a4b6 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-97ba394ab6aad2547f399ebbf757a4b6 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-73ee2d7b62e2aede20ca5de577cd7b7f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-73ee2d7b62e2aede20ca5de577cd7b7f diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc index 83be903e06482..5629a958479c7 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc @@ -1 +1 @@ -570 \ No newline at end of file +570 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 index 251ff85eda52d..1a9097317aa62 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -19,4 +19,4 @@ 5 5 5 5 8 8 -9 9 \ No newline at end of file +9 9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 index af8f457e93476..225fc24a2290f 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 @@ -19,4 +19,4 @@ val_5 val_5 val_5 val_5 val_5 val_5 val_8 val_8 -val_9 val_9 \ No newline at end of file +val_9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 index 251ff85eda52d..1a9097317aa62 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -19,4 +19,4 @@ 5 5 5 5 8 8 -9 9 \ No newline at end of file +9 9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 index af8f457e93476..225fc24a2290f 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 @@ -19,4 +19,4 @@ val_5 val_5 val_5 val_5 val_5 val_5 val_8 val_8 -val_9 val_9 \ No newline at end of file +val_9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 index 251ff85eda52d..1a9097317aa62 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -19,4 +19,4 @@ 5 5 5 5 8 8 -9 9 \ No newline at end of file +9 9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 index af8f457e93476..225fc24a2290f 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 @@ -19,4 +19,4 @@ val_5 val_5 val_5 val_5 val_5 val_5 val_8 val_8 -val_9 val_9 \ No newline at end of file +val_9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a index 1758dddccea2b..f5c89552bd3e6 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a @@ -1 +1 @@ -32 \ No newline at end of file +32 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd index bea0d09c49935..c92ba56847bf4 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd @@ -1 +1 @@ -207 \ No newline at end of file +207 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-0-24ca942f094b14b92086305cc125e833 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-0-24ca942f094b14b92086305cc125e833 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-1-16367c381d4b189b3640c92511244bfe similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-1-16367c381d4b189b3640c92511244bfe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-10-4706d21b17f993cc1cc94be6b7e04c28 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-10-4706d21b17f993cc1cc94be6b7e04c28 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-11-5e81d0b41cc58d8cc151046c7a111411 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-11-5e81d0b41cc58d8cc151046c7a111411 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-12-e8a77ff790bfd6740489fc4374ec6c3d similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-12-e8a77ff790bfd6740489fc4374ec6c3d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-13-920818d557d2525dabb5c539a80a1bbb similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-13-920818d557d2525dabb5c539a80a1bbb diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-14-455dfeeba27ecf53923db0cbf0aab908 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-14-455dfeeba27ecf53923db0cbf0aab908 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-15-11f98f575685beedc14a88fc47a61041 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-15-11f98f575685beedc14a88fc47a61041 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b new file mode 100644 index 0000000000000..2da41004e8914 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b @@ -0,0 +1,24 @@ +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +374 val_374 val_374 day1 1 +374 val_374 val_374 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-2-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-2-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-3-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-3-365488a703b0640acda73a7d7e6efa06 diff --git a/sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-4-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-4-d0ec6d66ff349db09fd455eec149efdb diff --git a/sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-5-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-5-3b0f76816be2c1b18a2058027a19bc9f diff --git a/sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-6-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-6-86473a0498e4361e4db0b4a22f2e8571 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-7-7e87a31677022b6a1c360922ef74754e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-7-7e87a31677022b6a1c360922ef74754e diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-9-ae5880516ea2f924cfbaeb919adc86e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-9-ae5880516ea2f924cfbaeb919adc86e diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-aa6ec7f17b48cf49f02ab7367453ab39 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-aa6ec7f17b48cf49f02ab7367453ab39 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-66b07c93d79ed9958b8427dad16c3ef3 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-66b07c93d79ed9958b8427dad16c3ef3 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-45b63361c1e5178b69a1531b238c8460 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-45b63361c1e5178b69a1531b238c8460 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-aa6ec7f17b48cf49f02ab7367453ab39 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-aa6ec7f17b48cf49f02ab7367453ab39 diff --git a/sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-66b07c93d79ed9958b8427dad16c3ef3 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-66b07c93d79ed9958b8427dad16c3ef3 diff --git a/sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1 diff --git a/sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-45b63361c1e5178b69a1531b238c8460 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-45b63361c1e5178b69a1531b238c8460 diff --git a/sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-ecca12a2f377c18c53563a534e9dd5a5 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-ecca12a2f377c18c53563a534e9dd5a5 diff --git a/sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-65930e1b01da720cf296ca3df668b58d similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-65930e1b01da720cf296ca3df668b58d diff --git a/sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fdbb11a2de2777dfdd916b59764d5c8e similarity index 100% rename from sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fdbb11a2de2777dfdd916b59764d5c8e diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 index dec2bf5d6199c..d6b24041cf041 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -19 \ No newline at end of file +19 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 index dec2bf5d6199c..d6b24041cf041 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -19 \ No newline at end of file +19 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed similarity index 100% rename from sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 index dec2bf5d6199c..d6b24041cf041 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -19 \ No newline at end of file +19 diff --git a/sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-9878d6ab0fe143611c88ffc0602197e3 similarity index 100% rename from sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-9878d6ab0fe143611c88ffc0602197e3 diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-a576657b9fd1c23aed85a7409887b2fb similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-a576657b9fd1c23aed85a7409887b2fb diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-350b202868590b5edaed18007fd5cbbe similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-350b202868590b5edaed18007fd5cbbe diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-1155788c7c133a73c3609e8052accfa5 similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-1155788c7c133a73c3609e8052accfa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-aa6ec7f17b48cf49f02ab7367453ab39 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-aa6ec7f17b48cf49f02ab7367453ab39 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-66b07c93d79ed9958b8427dad16c3ef3 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-66b07c93d79ed9958b8427dad16c3ef3 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-45b63361c1e5178b69a1531b238c8460 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-45b63361c1e5178b69a1531b238c8460 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-ecca12a2f377c18c53563a534e9dd5a5 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-ecca12a2f377c18c53563a534e9dd5a5 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-65930e1b01da720cf296ca3df668b58d similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-65930e1b01da720cf296ca3df668b58d diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-5cba470fbd02e730781a3b63fd9aa3e2 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-5cba470fbd02e730781a3b63fd9aa3e2 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-337e909605c780d00ad8895686defa06 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-337e909605c780d00ad8895686defa06 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-45b63361c1e5178b69a1531b238c8460 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-45b63361c1e5178b69a1531b238c8460 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 index 4a9735f855f96..ec7496a567609 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 @@ -3,4 +3,4 @@ 4 1 5 9 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 index 62f9457511f87..1e8b314962144 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -1 +1 @@ -6 \ No newline at end of file +6 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 index 2eafac63a9a98..2a51623eae15c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 @@ -3,4 +3,4 @@ 4 1 1 5 9 9 8 1 1 -9 1 1 \ No newline at end of file +9 1 1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce index 2ebc6516c7df1..f6b91e0e1f8dd 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce @@ -1 +1 @@ -56 \ No newline at end of file +56 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 index 4a9735f855f96..ec7496a567609 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 @@ -3,4 +3,4 @@ 4 1 5 9 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 index 62f9457511f87..1e8b314962144 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -1 +1 @@ -6 \ No newline at end of file +6 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 index 2eafac63a9a98..2a51623eae15c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 @@ -3,4 +3,4 @@ 4 1 1 5 9 9 8 1 1 -9 1 1 \ No newline at end of file +9 1 1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce index 2ebc6516c7df1..f6b91e0e1f8dd 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce @@ -1 +1 @@ -56 \ No newline at end of file +56 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index d0eea8a3ca661..0000000000000 --- a/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary None -ba_val binary None - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426673, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426673}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index 0a3c9f383a6ef..0000000000000 --- a/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary from deserializer -ba_val binary from deserializer - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426674, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:from deserializer), FieldSchema(name:ba_val, type:binary, comment:from deserializer)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426674, transient_lastDdlTime=1392426674}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index 66d33b789f1a7..0000000000000 --- a/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary None -ba_val binary None - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426676, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426676}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 b/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 deleted file mode 100644 index dcd1d8643e3cb..0000000000000 --- a/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 +++ /dev/null @@ -1 +0,0 @@ -2 val_2 diff --git a/sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/binary_constant-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c rename to sql/hive/src/test/resources/golden/binary_constant-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 b/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 deleted file mode 100644 index 2e65efe2a145d..0000000000000 --- a/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 +++ /dev/null @@ -1 +0,0 @@ -a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 b/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 new file mode 100644 index 0000000000000..78981922613b2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 @@ -0,0 +1 @@ +a diff --git a/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 b/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 deleted file mode 100644 index c5c8d29fdd13e..0000000000000 --- a/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 +++ /dev/null @@ -1,500 +0,0 @@ -238 val_238 -86 val_86 -311 val_311 -27 val_27 -165 val_165 -409 val_409 -255 val_255 -278 val_278 -98 val_98 -484 val_484 -265 val_265 -193 val_193 -401 val_401 -150 val_150 -273 val_273 -224 val_224 -369 val_369 -66 val_66 -128 val_128 -213 val_213 -146 val_146 -406 val_406 -429 val_429 -374 val_374 -152 val_152 -469 val_469 -145 val_145 -495 val_495 -37 val_37 -327 val_327 -281 val_281 -277 val_277 -209 val_209 -15 val_15 -82 val_82 -403 val_403 -166 val_166 -417 val_417 -430 val_430 -252 val_252 -292 val_292 -219 val_219 -287 val_287 -153 val_153 -193 val_193 -338 val_338 -446 val_446 -459 val_459 -394 val_394 -237 val_237 -482 val_482 -174 val_174 -413 val_413 -494 val_494 -207 val_207 -199 val_199 -466 val_466 -208 val_208 -174 val_174 -399 val_399 -396 val_396 -247 val_247 -417 val_417 -489 val_489 -162 val_162 -377 val_377 -397 val_397 -309 val_309 -365 val_365 -266 val_266 -439 val_439 -342 val_342 -367 val_367 -325 val_325 -167 val_167 -195 val_195 -475 val_475 -17 val_17 -113 val_113 -155 val_155 -203 val_203 -339 val_339 -0 val_0 -455 val_455 -128 val_128 -311 val_311 -316 val_316 -57 val_57 -302 val_302 -205 val_205 -149 val_149 -438 val_438 -345 val_345 -129 val_129 -170 val_170 -20 val_20 -489 val_489 -157 val_157 -378 val_378 -221 val_221 -92 val_92 -111 val_111 -47 val_47 -72 val_72 -4 val_4 -280 val_280 -35 val_35 -427 val_427 -277 val_277 -208 val_208 -356 val_356 -399 val_399 -169 val_169 -382 val_382 -498 val_498 -125 val_125 -386 val_386 -437 val_437 -469 val_469 -192 val_192 -286 val_286 -187 val_187 -176 val_176 -54 val_54 -459 val_459 -51 val_51 -138 val_138 -103 val_103 -239 val_239 -213 val_213 -216 val_216 -430 val_430 -278 val_278 -176 val_176 -289 val_289 -221 val_221 -65 val_65 -318 val_318 -332 val_332 -311 val_311 -275 val_275 -137 val_137 -241 val_241 -83 val_83 -333 val_333 -180 val_180 -284 val_284 -12 val_12 -230 val_230 -181 val_181 -67 val_67 -260 val_260 -404 val_404 -384 val_384 -489 val_489 -353 val_353 -373 val_373 -272 val_272 -138 val_138 -217 val_217 -84 val_84 -348 val_348 -466 val_466 -58 val_58 -8 val_8 -411 val_411 -230 val_230 -208 val_208 -348 val_348 -24 val_24 -463 val_463 -431 val_431 -179 val_179 -172 val_172 -42 val_42 -129 val_129 -158 val_158 -119 val_119 -496 val_496 -0 val_0 -322 val_322 -197 val_197 -468 val_468 -393 val_393 -454 val_454 -100 val_100 -298 val_298 -199 val_199 -191 val_191 -418 val_418 -96 val_96 -26 val_26 -165 val_165 -327 val_327 -230 val_230 -205 val_205 -120 val_120 -131 val_131 -51 val_51 -404 val_404 -43 val_43 -436 val_436 -156 val_156 -469 val_469 -468 val_468 -308 val_308 -95 val_95 -196 val_196 -288 val_288 -481 val_481 -457 val_457 -98 val_98 -282 val_282 -197 val_197 -187 val_187 -318 val_318 -318 val_318 -409 val_409 -470 val_470 -137 val_137 -369 val_369 -316 val_316 -169 val_169 -413 val_413 -85 val_85 -77 val_77 -0 val_0 -490 val_490 -87 val_87 -364 val_364 -179 val_179 -118 val_118 -134 val_134 -395 val_395 -282 val_282 -138 val_138 -238 val_238 -419 val_419 -15 val_15 -118 val_118 -72 val_72 -90 val_90 -307 val_307 -19 val_19 -435 val_435 -10 val_10 -277 val_277 -273 val_273 -306 val_306 -224 val_224 -309 val_309 -389 val_389 -327 val_327 -242 val_242 -369 val_369 -392 val_392 -272 val_272 -331 val_331 -401 val_401 -242 val_242 -452 val_452 -177 val_177 -226 val_226 -5 val_5 -497 val_497 -402 val_402 -396 val_396 -317 val_317 -395 val_395 -58 val_58 -35 val_35 -336 val_336 -95 val_95 -11 val_11 -168 val_168 -34 val_34 -229 val_229 -233 val_233 -143 val_143 -472 val_472 -322 val_322 -498 val_498 -160 val_160 -195 val_195 -42 val_42 -321 val_321 -430 val_430 -119 val_119 -489 val_489 -458 val_458 -78 val_78 -76 val_76 -41 val_41 -223 val_223 -492 val_492 -149 val_149 -449 val_449 -218 val_218 -228 val_228 -138 val_138 -453 val_453 -30 val_30 -209 val_209 -64 val_64 -468 val_468 -76 val_76 -74 val_74 -342 val_342 -69 val_69 -230 val_230 -33 val_33 -368 val_368 -103 val_103 -296 val_296 -113 val_113 -216 val_216 -367 val_367 -344 val_344 -167 val_167 -274 val_274 -219 val_219 -239 val_239 -485 val_485 -116 val_116 -223 val_223 -256 val_256 -263 val_263 -70 val_70 -487 val_487 -480 val_480 -401 val_401 -288 val_288 -191 val_191 -5 val_5 -244 val_244 -438 val_438 -128 val_128 -467 val_467 -432 val_432 -202 val_202 -316 val_316 -229 val_229 -469 val_469 -463 val_463 -280 val_280 -2 val_2 -35 val_35 -283 val_283 -331 val_331 -235 val_235 -80 val_80 -44 val_44 -193 val_193 -321 val_321 -335 val_335 -104 val_104 -466 val_466 -366 val_366 -175 val_175 -403 val_403 -483 val_483 -53 val_53 -105 val_105 -257 val_257 -406 val_406 -409 val_409 -190 val_190 -406 val_406 -401 val_401 -114 val_114 -258 val_258 -90 val_90 -203 val_203 -262 val_262 -348 val_348 -424 val_424 -12 val_12 -396 val_396 -201 val_201 -217 val_217 -164 val_164 -431 val_431 -454 val_454 -478 val_478 -298 val_298 -125 val_125 -431 val_431 -164 val_164 -424 val_424 -187 val_187 -382 val_382 -5 val_5 -70 val_70 -397 val_397 -480 val_480 -291 val_291 -24 val_24 -351 val_351 -255 val_255 -104 val_104 -70 val_70 -163 val_163 -438 val_438 -119 val_119 -414 val_414 -200 val_200 -491 val_491 -237 val_237 -439 val_439 -360 val_360 -248 val_248 -479 val_479 -305 val_305 -417 val_417 -199 val_199 -444 val_444 -120 val_120 -429 val_429 -169 val_169 -443 val_443 -323 val_323 -325 val_325 -277 val_277 -230 val_230 -478 val_478 -178 val_178 -468 val_468 -310 val_310 -317 val_317 -333 val_333 -493 val_493 -460 val_460 -207 val_207 -249 val_249 -265 val_265 -480 val_480 -83 val_83 -136 val_136 -353 val_353 -172 val_172 -214 val_214 -462 val_462 -233 val_233 -406 val_406 -133 val_133 -175 val_175 -189 val_189 -454 val_454 -375 val_375 -401 val_401 -421 val_421 -407 val_407 -384 val_384 -256 val_256 -26 val_26 -134 val_134 -67 val_67 -384 val_384 -379 val_379 -18 val_18 -462 val_462 -492 val_492 -100 val_100 -298 val_298 -9 val_9 -341 val_341 -498 val_498 -146 val_146 -458 val_458 -362 val_362 -186 val_186 -285 val_285 -348 val_348 -167 val_167 -18 val_18 -273 val_273 -183 val_183 -281 val_281 -344 val_344 -97 val_97 -469 val_469 -315 val_315 -84 val_84 -28 val_28 -37 val_37 -448 val_448 -152 val_152 -348 val_348 -307 val_307 -194 val_194 -414 val_414 -477 val_477 -222 val_222 -126 val_126 -90 val_90 -169 val_169 -403 val_403 -400 val_400 -200 val_200 -97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index df14abbcc193f..0000000000000 --- a/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary from deserializer -ba_val binary from deserializer - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426678, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:from deserializer), FieldSchema(name:ba_val, type:binary, comment:from deserializer)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426678, transient_lastDdlTime=1392426678}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index 0c246ae33e56f..0000000000000 --- a/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary from deserializer -ba_val binary from deserializer - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426679, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426679}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 b/sql/hive/src/test/resources/golden/binarysortable_1-1-aca7ae366662c9698b5d2c01a6cb3948 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 rename to sql/hive/src/test/resources/golden/binarysortable_1-1-aca7ae366662c9698b5d2c01a6cb3948 diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 b/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 index 27687b47813a9..d6ee76110e8a9 100644 --- a/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 +++ b/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 @@ -7,4 +7,4 @@ a^@bc^A^B^A^@ 1.0 test^@^@^A^Atest 6.0 test^@test 4.0 -test^Atest 5.0 \ No newline at end of file +test^Atest 5.0 diff --git a/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a b/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a index 9bdd310949be8..b62097939c833 100644 --- a/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a +++ b/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a @@ -1 +1 @@ -5 5.0 5.0 5.0 5 true 1 \ No newline at end of file +5 5.0 5.0 5.0 5 true 1 diff --git a/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 b/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 deleted file mode 100644 index bbe268ea91ddf..0000000000000 --- a/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 +++ /dev/null @@ -1 +0,0 @@ -1.0 1.4 1.6 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 NULL 2147483647 -2147483648 32767 -32768 -128 127 NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/char_varchar_udf-0-4de1b2fa0908a3d856474aae2bc38c08 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 rename to sql/hive/src/test/resources/golden/char_varchar_udf-0-4de1b2fa0908a3d856474aae2bc38c08 diff --git a/sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/char_varchar_udf-1-5b1e7c580ed5e756d30a4c557af8902 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 rename to sql/hive/src/test/resources/golden/char_varchar_udf-1-5b1e7c580ed5e756d30a4c557af8902 diff --git a/sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/char_varchar_udf-2-880ce74a83bb2bfb9c9bd584b8874ac similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 rename to sql/hive/src/test/resources/golden/char_varchar_udf-2-880ce74a83bb2bfb9c9bd584b8874ac diff --git a/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 b/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 index 87d0f8dd52b68..ae8b7551d9630 100644 --- a/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 +++ b/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 @@ -1 +1 @@ -10 val_10 \ No newline at end of file +10 val_10 diff --git a/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e b/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e +++ b/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 b/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 +++ b/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 b/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 +++ b/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 b/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 index 1a018b640eb6f..e2ef71e141c46 100644 --- a/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 +++ b/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 @@ -1 +1 @@ -20 val_20 20 \ No newline at end of file +20 val_20 20 diff --git a/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 b/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 index 333a4cf9af123..5b04be3cdf046 100644 --- a/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 +++ b/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 @@ -1 +1 @@ -20 val_20 20 val_20 \ No newline at end of file +20 val_20 20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f b/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f index 333a4cf9af123..5b04be3cdf046 100644 --- a/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f +++ b/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f @@ -1 +1 @@ -20 val_20 20 val_20 \ No newline at end of file +20 val_20 20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e b/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e index 1a018b640eb6f..e2ef71e141c46 100644 --- a/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e +++ b/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e @@ -1 +1 @@ -20 val_20 20 \ No newline at end of file +20 val_20 20 diff --git a/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a b/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a index a79654385b09f..764cbe5f62b60 100644 --- a/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a +++ b/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a @@ -495,4 +495,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f b/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f +++ b/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 b/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 +++ b/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 b/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 +++ b/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 b/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 +++ b/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f b/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f deleted file mode 100644 index 15e92afeeca27..0000000000000 --- a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] ["0","0","0"] {"key_0":"value_0"} 1712634731 record_0 -[1,2,3] ["10","100","1000"] {"key_1":"value_1"} 465985200 record_1 -[2,4,6] ["20","200","2000"] {"key_2":"value_2"} -751827638 record_2 -[3,6,9] ["30","300","3000"] {"key_3":"value_3"} 477111222 record_3 -[4,8,12] ["40","400","4000"] {"key_4":"value_4"} -734328909 record_4 -[5,10,15] ["50","500","5000"] {"key_5":"value_5"} -1952710710 record_5 -[6,12,18] ["60","600","6000"] {"key_6":"value_6"} 1244525190 record_6 -[7,14,21] ["70","700","7000"] {"key_7":"value_7"} -1461153973 record_7 -[8,16,24] ["80","800","8000"] {"key_8":"value_8"} 1638581578 record_8 -[9,18,27] ["90","900","9000"] {"key_9":"value_9"} 336964413 record_9 -NULL NULL NULL 0 NULL diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 b/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 deleted file mode 100644 index 6ad4db1788424..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 +++ /dev/null @@ -1,58 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 2000.0))) (TOK_TABCOLNAME employeeID)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - employee_part - TableScan - alias: employee_part - Select Operator - expressions: - expr: employeeid - type: int - outputColumnNames: employeeid - Group By Operator - aggregations: - expr: compute_stats(employeeid, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: - expr: _col0 - type: struct - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: employeeID - Column Types: int - Partition: employeesalary=2000.0 - Table: Employee_Part - diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 b/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 deleted file mode 100644 index 91ce2a521cde1..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 +++ /dev/null @@ -1,129 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 2000.0))) (TOK_TABCOLNAME employeeID)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - employee_part - TableScan - alias: employee_part - GatherStats: false - Select Operator - expressions: - expr: employeeid - type: int - outputColumnNames: employeeid - Group By Operator - aggregations: - expr: compute_stats(employeeid, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - Path -> Alias: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0 [employee_part] - Path -> Partition: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0 - Partition - base file name: employeesalary=2000.0 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - employeesalary 2000.0 - properties: - bucket_count -1 - columns employeeid,employeename - columns.types int:string - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0 - name default.employee_part - numFiles 1 - numRows 0 - partition_columns employeesalary - rawDataSize 0 - serialization.ddl struct employee_part { i32 employeeid, string employeename} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 105 - transient_lastDdlTime 1389728706 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns employeeid,employeename - columns.types int:string - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part - name default.employee_part - numFiles 2 - numPartitions 2 - numRows 0 - partition_columns employeesalary - rawDataSize 0 - serialization.ddl struct employee_part { i32 employeeid, string employeename} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 210 - transient_lastDdlTime 1389728706 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.employee_part - name: default.employee_part - Truncated Path -> Alias: - /employee_part/employeesalary=2000.0 [employee_part] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: - expr: _col0 - type: struct - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 - directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-07_241_106202206012377173-1/-ext-10001 - NumFilesPerFileSink: 1 - Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-07_241_106202206012377173-1/-ext-10001/ - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - columns _col0 - columns.types struct - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: employeeID - Column Types: int - Partition: employeesalary=2000.0 - Table: Employee_Part - Is Table Level Stats: false - diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 b/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 deleted file mode 100644 index 777024f6946e3..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 +++ /dev/null @@ -1,58 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 4000.0))) (TOK_TABCOLNAME employeeID)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - employee_part - TableScan - alias: employee_part - Select Operator - expressions: - expr: employeeid - type: int - outputColumnNames: employeeid - Group By Operator - aggregations: - expr: compute_stats(employeeid, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: - expr: _col0 - type: struct - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: employeeID - Column Types: int - Partition: employeesalary=4000.0 - Table: Employee_Part - diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce b/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce deleted file mode 100644 index cd72c7efbf56f..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce +++ /dev/null @@ -1,129 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 4000.0))) (TOK_TABCOLNAME employeeID)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - employee_part - TableScan - alias: employee_part - GatherStats: false - Select Operator - expressions: - expr: employeeid - type: int - outputColumnNames: employeeid - Group By Operator - aggregations: - expr: compute_stats(employeeid, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - Path -> Alias: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0 [employee_part] - Path -> Partition: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0 - Partition - base file name: employeesalary=4000.0 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - employeesalary 4000.0 - properties: - bucket_count -1 - columns employeeid,employeename - columns.types int:string - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0 - name default.employee_part - numFiles 1 - numRows 0 - partition_columns employeesalary - rawDataSize 0 - serialization.ddl struct employee_part { i32 employeeid, string employeename} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 105 - transient_lastDdlTime 1389728706 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns employeeid,employeename - columns.types int:string - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part - name default.employee_part - numFiles 2 - numPartitions 2 - numRows 0 - partition_columns employeesalary - rawDataSize 0 - serialization.ddl struct employee_part { i32 employeeid, string employeename} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 210 - transient_lastDdlTime 1389728706 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.employee_part - name: default.employee_part - Truncated Path -> Alias: - /employee_part/employeesalary=4000.0 [employee_part] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: - expr: _col0 - type: struct - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 - directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-24_849_6968895828655634809-1/-ext-10001 - NumFilesPerFileSink: 1 - Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-24_849_6968895828655634809-1/-ext-10001/ - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - columns _col0 - columns.types struct - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: employeeID - Column Types: int - Partition: employeesalary=4000.0 - Table: Employee_Part - Is Table Level Stats: false - diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 deleted file mode 100644 index d1e5e7375467d..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 +++ /dev/null @@ -1,73 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME UserVisits_web_text_none)) (TOK_TABCOLNAME sourceIP avgTimeOnSite adRevenue)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - uservisits_web_text_none - TableScan - alias: uservisits_web_text_none - Select Operator - expressions: - expr: sourceip - type: string - expr: avgtimeonsite - type: int - expr: adrevenue - type: float - outputColumnNames: sourceip, avgtimeonsite, adrevenue - Group By Operator - aggregations: - expr: compute_stats(sourceip, 16) - expr: compute_stats(avgtimeonsite, 16) - expr: compute_stats(adrevenue, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0, _col1, _col2 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - expr: compute_stats(VALUE._col1) - expr: compute_stats(VALUE._col2) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0, _col1, _col2 - Select Operator - expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: sourceIP, avgTimeOnSite, adRevenue - Column Types: string, int, float - Table: UserVisits_web_text_none - diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 deleted file mode 100644 index 3f3aa581b43f9..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 +++ /dev/null @@ -1,141 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME UserVisits_web_text_none)) (TOK_TABCOLNAME sourceIP avgTimeOnSite adRevenue)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - uservisits_web_text_none - TableScan - alias: uservisits_web_text_none - GatherStats: false - Select Operator - expressions: - expr: sourceip - type: string - expr: avgtimeonsite - type: int - expr: adrevenue - type: float - outputColumnNames: sourceip, avgtimeonsite, adrevenue - Group By Operator - aggregations: - expr: compute_stats(sourceip, 16) - expr: compute_stats(avgtimeonsite, 16) - expr: compute_stats(adrevenue, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0, _col1, _col2 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - Path -> Alias: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none [uservisits_web_text_none] - Path -> Partition: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none - Partition - base file name: uservisits_web_text_none - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns sourceip,desturl,visitdate,adrevenue,useragent,ccode,lcode,skeyword,avgtimeonsite - columns.types string:string:string:float:string:string:string:string:int - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none - name default.uservisits_web_text_none - numFiles 1 - numPartitions 0 - numRows 0 - rawDataSize 0 - serialization.ddl struct uservisits_web_text_none { string sourceip, string desturl, string visitdate, float adrevenue, string useragent, string ccode, string lcode, string skeyword, i32 avgtimeonsite} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 7060 - transient_lastDdlTime 1389728748 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns sourceip,desturl,visitdate,adrevenue,useragent,ccode,lcode,skeyword,avgtimeonsite - columns.types string:string:string:float:string:string:string:string:int - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none - name default.uservisits_web_text_none - numFiles 1 - numPartitions 0 - numRows 0 - rawDataSize 0 - serialization.ddl struct uservisits_web_text_none { string sourceip, string desturl, string visitdate, float adrevenue, string useragent, string ccode, string lcode, string skeyword, i32 avgtimeonsite} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 7060 - transient_lastDdlTime 1389728748 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.uservisits_web_text_none - name: default.uservisits_web_text_none - Truncated Path -> Alias: - /uservisits_web_text_none [uservisits_web_text_none] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - expr: compute_stats(VALUE._col1) - expr: compute_stats(VALUE._col2) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0, _col1, _col2 - Select Operator - expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - GlobalTableId: 0 - directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-49_160_8862102294255849057-1/-ext-10001 - NumFilesPerFileSink: 1 - Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-49_160_8862102294255849057-1/-ext-10001/ - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - columns _col0,_col1,_col2 - columns.types struct:struct:struct - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: sourceIP, avgTimeOnSite, adRevenue - Column Types: string, int, float - Table: UserVisits_web_text_none - Is Table Level Stats: true - diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d b/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d deleted file mode 100644 index 4ff444febde63..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d +++ /dev/null @@ -1,89 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME empty_tab)) (TOK_TABCOLNAME a b c d e)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - empty_tab - TableScan - alias: empty_tab - Select Operator - expressions: - expr: a - type: int - expr: b - type: double - expr: c - type: string - expr: d - type: boolean - expr: e - type: binary - outputColumnNames: a, b, c, d, e - Group By Operator - aggregations: - expr: compute_stats(a, 16) - expr: compute_stats(b, 16) - expr: compute_stats(c, 16) - expr: compute_stats(d, 16) - expr: compute_stats(e, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - expr: _col3 - type: struct - expr: _col4 - type: struct - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - expr: compute_stats(VALUE._col1) - expr: compute_stats(VALUE._col2) - expr: compute_stats(VALUE._col3) - expr: compute_stats(VALUE._col4) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Select Operator - expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - expr: _col3 - type: struct - expr: _col4 - type: struct - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: a, b, c, d, e - Column Types: int, double, string, boolean, binary - Table: empty_tab - diff --git a/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 +++ b/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c +++ b/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf +++ b/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 +++ b/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 +++ b/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea +++ b/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 b/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 +++ b/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 b/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 index 8f8e1f4b21fe3..b40eac432935c 100644 --- a/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 +++ b/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 @@ -497,4 +497,4 @@ 97 val_97 97 val_97 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 deleted file mode 100644 index 80fa68b84c17e..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 +++ /dev/null @@ -1,8 +0,0 @@ -value=2010-04-21 09%3A45%3A00 -value=val_0 -value=val_2 -value=val_4 -value=val_5 -value=val_8 -value=val_9 -value=| \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c deleted file mode 100644 index ff6141674e603..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c +++ /dev/null @@ -1,12 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -11 | -19 2010-04-21 09:45:00 -2 val_2 -4 val_4 -5 val_5 -5 val_5 -5 val_5 -8 val_8 -9 val_9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 deleted file mode 100644 index 3cacc0b93c9c9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 +++ /dev/null @@ -1 +0,0 @@ -12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 deleted file mode 100644 index 1a0aa74952afa..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 +++ /dev/null @@ -1,2 +0,0 @@ -2008-04-08 1000 -2008-04-09 1000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 deleted file mode 100644 index 80fa68b84c17e..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 +++ /dev/null @@ -1,8 +0,0 @@ -value=2010-04-21 09%3A45%3A00 -value=val_0 -value=val_2 -value=val_4 -value=val_5 -value=val_8 -value=val_9 -value=| \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c deleted file mode 100644 index ff6141674e603..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c +++ /dev/null @@ -1,12 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -11 | -19 2010-04-21 09:45:00 -2 val_2 -4 val_4 -5 val_5 -5 val_5 -5 val_5 -8 val_8 -9 val_9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 deleted file mode 100644 index 3cacc0b93c9c9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 +++ /dev/null @@ -1 +0,0 @@ -12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 deleted file mode 100644 index 1a0aa74952afa..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 +++ /dev/null @@ -1,2 +0,0 @@ -2008-04-08 1000 -2008-04-09 1000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 b/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 deleted file mode 100644 index 80665a4d4c983..0000000000000 --- a/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 +++ /dev/null @@ -1,12 +0,0 @@ -key int None -value string None -ds string None -hr string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -Detailed Partition Information Partition(values:[2010-08-03, 00], dbName:default, tableName:combine_3_srcpart_seq_rc, createTime:1390898644, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/combine_3_srcpart_seq_rc/ds=2010-08-03/hr=00, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390898644, numRows=500, totalSize=15250, rawDataSize=5312}) diff --git a/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b b/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b deleted file mode 100644 index 5a87a3aec7cf5..0000000000000 --- a/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b +++ /dev/null @@ -1,12 +0,0 @@ -key int from deserializer -value string from deserializer -ds string None -hr string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -Detailed Partition Information Partition(values:[2010-08-03, 001], dbName:default, tableName:combine_3_srcpart_seq_rc, createTime:1390898653, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/combine_3_srcpart_seq_rc/ds=2010-08-03/hr=001, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390898653, numRows=500, totalSize=2202, rawDataSize=4551}) diff --git a/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc b/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc deleted file mode 100644 index 7b6455db7834b..0000000000000 --- a/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc +++ /dev/null @@ -1,30 +0,0 @@ -0 val_0 2010-08-03 00 -0 val_0 2010-08-03 00 -0 val_0 2010-08-03 00 -0 val_0 2010-08-03 001 -0 val_0 2010-08-03 001 -0 val_0 2010-08-03 001 -2 val_2 2010-08-03 00 -2 val_2 2010-08-03 001 -4 val_4 2010-08-03 00 -4 val_4 2010-08-03 001 -5 val_5 2010-08-03 00 -5 val_5 2010-08-03 00 -5 val_5 2010-08-03 00 -5 val_5 2010-08-03 001 -5 val_5 2010-08-03 001 -5 val_5 2010-08-03 001 -8 val_8 2010-08-03 00 -8 val_8 2010-08-03 001 -9 val_9 2010-08-03 00 -9 val_9 2010-08-03 001 -10 val_10 2010-08-03 00 -10 val_10 2010-08-03 001 -11 val_11 2010-08-03 00 -11 val_11 2010-08-03 001 -12 val_12 2010-08-03 00 -12 val_12 2010-08-03 00 -12 val_12 2010-08-03 001 -12 val_12 2010-08-03 001 -15 val_15 2010-08-03 00 -15 val_15 2010-08-03 00 diff --git a/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 b/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 deleted file mode 100644 index d57cb5369e219..0000000000000 --- a/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 +++ /dev/null @@ -1,30 +0,0 @@ -0 1 -0 1 -0 1 -0 11 -0 11 -0 11 -2 1 -2 11 -4 1 -4 11 -8 1 -8 11 -10 1 -10 11 -12 1 -12 1 -12 11 -12 11 -18 1 -18 1 -18 11 -18 11 -20 1 -20 11 -24 1 -24 1 -24 11 -24 11 -26 1 -26 1 diff --git a/sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af b/sql/hive/src/test/resources/golden/compute_stats_binary-1-44b15c6c9600109e064a5ea5f9c81051 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af rename to sql/hive/src/test/resources/golden/compute_stats_binary-1-44b15c6c9600109e064a5ea5f9c81051 diff --git a/sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 b/sql/hive/src/test/resources/golden/compute_stats_boolean-1-72ee4bdd5cea69136940dc40e6890e1d similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 rename to sql/hive/src/test/resources/golden/compute_stats_boolean-1-72ee4bdd5cea69136940dc40e6890e1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e b/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e index dc7b54ad01435..bb95160cb6e07 100644 --- a/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e +++ b/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e @@ -1 +1 @@ -33 \ No newline at end of file +33 diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d b/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d index dd487e6fea3ff..279805d381a21 100644 --- a/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d +++ b/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d @@ -1 +1 @@ -{"columntype":"Boolean","counttrues":13,"countfalses":19,"countnulls":1} \ No newline at end of file +{"columntype":"Boolean","counttrues":13,"countfalses":19,"countnulls":1} diff --git a/sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 b/sql/hive/src/test/resources/golden/compute_stats_double-1-8f634b9e334fd58e71844e6283d9794d similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 rename to sql/hive/src/test/resources/golden/compute_stats_double-1-8f634b9e334fd58e71844e6283d9794d diff --git a/sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a b/sql/hive/src/test/resources/golden/compute_stats_long-1-9313f166464633b3929707d7ef11d758 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a rename to sql/hive/src/test/resources/golden/compute_stats_long-1-9313f166464633b3929707d7ef11d758 diff --git a/sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b b/sql/hive/src/test/resources/golden/compute_stats_string-1-3491ef2747a8f34899108d4ae8ebc7eb similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b rename to sql/hive/src/test/resources/golden/compute_stats_string-1-3491ef2747a8f34899108d4ae8ebc7eb diff --git a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 index c615f0148ccd1..d35bf9093ca9c 100644 --- a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 +++ b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 @@ -19,9 +19,9 @@ my_stringset struct<> from deserializer my_enumset struct<> from deserializer my_structset struct<> from deserializer optionals struct<> from deserializer -b string None +b string # Partition Information # col_name data_type comment -b string None \ No newline at end of file +b string diff --git a/sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f b/sql/hive/src/test/resources/golden/correlationoptimizer1-26-5522db58d123d1bec48b6e71c1b258f3 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f rename to sql/hive/src/test/resources/golden/correlationoptimizer1-26-5522db58d123d1bec48b6e71c1b258f3 diff --git a/sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd b/sql/hive/src/test/resources/golden/correlationoptimizer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd rename to sql/hive/src/test/resources/golden/correlationoptimizer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f new file mode 100644 index 0000000000000..fb15947b3d0bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f @@ -0,0 +1,16 @@ +NULL NULL 10 +66 val_66 1 +98 val_98 2 +128 NULL 1 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 NULL 1 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 NULL 1 +401 val_401 5 +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933 b/sql/hive/src/test/resources/golden/correlationoptimizer1-35-e1d4857548e626bb31d70c096b8d0a95 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-35-e1d4857548e626bb31d70c096b8d0a95 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f new file mode 100644 index 0000000000000..fb15947b3d0bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f @@ -0,0 +1,16 @@ +NULL NULL 10 +66 val_66 1 +98 val_98 2 +128 NULL 1 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 NULL 1 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 NULL 1 +401 val_401 5 +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62 b/sql/hive/src/test/resources/golden/correlationoptimizer1-38-ef6502d6b282c8a6d228bba395b24724 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-38-ef6502d6b282c8a6d228bba395b24724 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-39-ea87e76dba02a46cb958148333e397b7 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-39-ea87e76dba02a46cb958148333e397b7 diff --git a/sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62 b/sql/hive/src/test/resources/golden/correlationoptimizer1-41-b79b220859c09354e23b533c105ccbab similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-41-b79b220859c09354e23b533c105ccbab diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-42-ea87e76dba02a46cb958148333e397b7 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-42-ea87e76dba02a46cb958148333e397b7 diff --git a/sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 b/sql/hive/src/test/resources/golden/correlationoptimizer1-44-638e5300f4c892c2bf27bd91a8f81b64 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-44-638e5300f4c892c2bf27bd91a8f81b64 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-45-66010469a9cdb66851da9a727ef9fdad similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad rename to sql/hive/src/test/resources/golden/correlationoptimizer1-45-66010469a9cdb66851da9a727ef9fdad diff --git a/sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b b/sql/hive/src/test/resources/golden/correlationoptimizer1-47-3514c74c7f68f2d70cc6d51ac46c20 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b rename to sql/hive/src/test/resources/golden/correlationoptimizer1-47-3514c74c7f68f2d70cc6d51ac46c20 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-48-66010469a9cdb66851da9a727ef9fdad similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad rename to sql/hive/src/test/resources/golden/correlationoptimizer1-48-66010469a9cdb66851da9a727ef9fdad diff --git a/sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 b/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-49-b9d963d24994c47c3776dda6f7d3881f diff --git a/sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 b/sql/hive/src/test/resources/golden/correlationoptimizer1-50-7490df6719cd7e47aa08dbcbc3266a92 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-50-7490df6719cd7e47aa08dbcbc3266a92 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-51-e71195e7d9f557e2abc7f03462d22dba similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba rename to sql/hive/src/test/resources/golden/correlationoptimizer1-51-e71195e7d9f557e2abc7f03462d22dba diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/correlationoptimizer1-52-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/correlationoptimizer1-52-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 b/sql/hive/src/test/resources/golden/correlationoptimizer1-53-73da9fe2b0c2ee26c021ec3f2fa27272 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-53-73da9fe2b0c2ee26c021ec3f2fa27272 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-54-e71195e7d9f557e2abc7f03462d22dba similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba rename to sql/hive/src/test/resources/golden/correlationoptimizer1-54-e71195e7d9f557e2abc7f03462d22dba diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/correlationoptimizer1-55-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/correlationoptimizer1-55-b1e2ade89ae898650f0be4f796d8947b diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 b/sql/hive/src/test/resources/golden/correlationoptimizer1-57-fcf9bcb522f542637ccdea863b408448 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-57-fcf9bcb522f542637ccdea863b408448 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-58-3070366869308907e54797927805603 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-58-3070366869308907e54797927805603 diff --git a/sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 b/sql/hive/src/test/resources/golden/correlationoptimizer1-60-dad56e1f06c808b29e5dc8fb0c49efb2 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-60-dad56e1f06c808b29e5dc8fb0c49efb2 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-61-3070366869308907e54797927805603 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-61-3070366869308907e54797927805603 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/correlationoptimizer1-62-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-62-b9d963d24994c47c3776dda6f7d3881f diff --git a/sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 b/sql/hive/src/test/resources/golden/correlationoptimizer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-64-a6bba6d9b422adb386b35c62cecb548 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-64-a6bba6d9b422adb386b35c62cecb548 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/correlationoptimizer1-65-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/correlationoptimizer1-65-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f b/sql/hive/src/test/resources/golden/correlationoptimizer1-66-d6bbaf0d40010159095e4cac025c50c5 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f rename to sql/hive/src/test/resources/golden/correlationoptimizer1-66-d6bbaf0d40010159095e4cac025c50c5 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-67-a6bba6d9b422adb386b35c62cecb548 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-67-a6bba6d9b422adb386b35c62cecb548 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 index 0b1ee37d7831c..6eebe80953bf0 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 @@ -34,4 +34,4 @@ 406 1 406 406 1 406 406 1 406 -406 1 406 \ No newline at end of file +406 1 406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 index 0b1ee37d7831c..6eebe80953bf0 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 @@ -34,4 +34,4 @@ 406 1 406 406 1 406 406 1 406 -406 1 406 \ No newline at end of file +406 1 406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 b/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 index 6c6dc691bbff2..e7c8cc75a0d6c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 @@ -1 +1 @@ -79136 500 3556 15 \ No newline at end of file +79136 500 3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 b/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 index f4bb720dfd7f8..96d1ad9dd0559 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 @@ -1 +1 @@ -3556 37 3556 25 \ No newline at end of file +3556 37 3556 25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 b/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 index f4bb720dfd7f8..96d1ad9dd0559 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 @@ -1 +1 @@ -3556 37 3556 25 \ No newline at end of file +3556 37 3556 25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e b/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e index 4acbb60e81661..716f95a30304b 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e @@ -1 +1 @@ -79136 500 3556 25 \ No newline at end of file +79136 500 3556 25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e b/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e index 4acbb60e81661..716f95a30304b 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e @@ -1 +1 @@ -79136 500 3556 25 \ No newline at end of file +79136 500 3556 25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 b/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 index a95fce80fd7b4..3821ee1926f17 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 @@ -1 +1 @@ -79136 310 \ No newline at end of file +79136 310 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d b/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d index c6243d7056353..7e1b7f7408e2d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d @@ -1 +1 @@ -3556 37 3556 15 \ No newline at end of file +3556 37 3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 b/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 index a95fce80fd7b4..3821ee1926f17 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 @@ -1 +1 @@ -79136 310 \ No newline at end of file +79136 310 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b b/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b index 556b77ecfc9eb..a8707661c9399 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b @@ -1 +1 @@ -9992 3531902962 9992 37 \ No newline at end of file +9992 3531902962 9992 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b b/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b index 556b77ecfc9eb..a8707661c9399 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b @@ -1 +1 @@ -9992 3531902962 9992 37 \ No newline at end of file +9992 3531902962 9992 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d b/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d index c6243d7056353..7e1b7f7408e2d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d @@ -1 +1 @@ -3556 37 3556 15 \ No newline at end of file +3556 37 3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 b/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 index 6c6dc691bbff2..e7c8cc75a0d6c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 @@ -1 +1 @@ -79136 500 3556 15 \ No newline at end of file +79136 500 3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 b/sql/hive/src/test/resources/golden/correlationoptimizer4-1-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 rename to sql/hive/src/test/resources/golden/correlationoptimizer4-1-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e index 14e309fdcad89..281e236e8a80d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -1 +1 @@ -13 10 \ No newline at end of file +13 10 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e index 14e309fdcad89..281e236e8a80d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -1 +1 @@ -13 10 \ No newline at end of file +13 10 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 b/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 index 1b8c137073e37..1ed2737b0d1fb 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 @@ -1 +1 @@ -22 12 \ No newline at end of file +22 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 b/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 index 1b8c137073e37..1ed2737b0d1fb 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 @@ -1 +1 @@ -22 12 \ No newline at end of file +22 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b b/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b index e6c95e600a2c0..35b3da1e4da5c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b @@ -1 +1 @@ -13 12 \ No newline at end of file +13 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf b/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf index 0248cc90cb2c6..7b6dfccea7a0c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf @@ -1 +1 @@ -21 12 \ No newline at end of file +21 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 b/sql/hive/src/test/resources/golden/correlationoptimizer4-3-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 rename to sql/hive/src/test/resources/golden/correlationoptimizer4-3-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf b/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf index 0248cc90cb2c6..7b6dfccea7a0c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf @@ -1 +1 @@ -21 12 \ No newline at end of file +21 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 b/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 index 0248cc90cb2c6..7b6dfccea7a0c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 @@ -1 +1 @@ -21 12 \ No newline at end of file +21 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 b/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 index fd3a42ebe67e7..60ee3e8737989 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 @@ -1 +1 @@ -21 14 \ No newline at end of file +21 14 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 b/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 index fd3a42ebe67e7..60ee3e8737989 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 @@ -1 +1 @@ -21 14 \ No newline at end of file +21 14 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab b/sql/hive/src/test/resources/golden/correlationoptimizer4-5-b5777cff7c522c4b527f77988e7f6bf1 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab rename to sql/hive/src/test/resources/golden/correlationoptimizer4-5-b5777cff7c522c4b527f77988e7f6bf1 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e index 14e309fdcad89..281e236e8a80d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -1 +1 @@ -13 10 \ No newline at end of file +13 10 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 index 19b8a2aea8f64..8f9dae31cc51c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 @@ -12,4 +12,4 @@ 311 1 311 9 369 1 369 9 401 1 401 25 -406 1 406 16 \ No newline at end of file +406 1 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db b/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db index c4a418f59625b..26151f7b6d0f1 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db @@ -12,4 +12,4 @@ 311 311 3 369 369 3 401 401 5 -406 406 4 \ No newline at end of file +406 406 4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db b/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db index c4a418f59625b..26151f7b6d0f1 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db @@ -12,4 +12,4 @@ 311 311 3 369 369 3 401 401 5 -406 406 4 \ No newline at end of file +406 406 4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de index 9c8189500649e..c7d10af90e353 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de @@ -12,4 +12,4 @@ 311 311 9 369 369 9 401 401 25 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de index 9c8189500649e..c7d10af90e353 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de @@ -12,4 +12,4 @@ 311 311 9 369 369 9 401 401 25 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 index 0b1ee37d7831c..6eebe80953bf0 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 @@ -34,4 +34,4 @@ 406 1 406 406 1 406 406 1 406 -406 1 406 \ No newline at end of file +406 1 406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 index 19b8a2aea8f64..8f9dae31cc51c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 @@ -12,4 +12,4 @@ 311 1 311 9 369 1 369 9 401 1 401 25 -406 1 406 16 \ No newline at end of file +406 1 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 index 0b1ee37d7831c..6eebe80953bf0 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 @@ -34,4 +34,4 @@ 406 1 406 406 1 406 406 1 406 -406 1 406 \ No newline at end of file +406 1 406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 b/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 index 7c591d6146fd6..4e3bbb2779bf3 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 @@ -34,4 +34,4 @@ 406 406 16 406 406 16 406 406 16 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 b/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 index 7c591d6146fd6..4e3bbb2779bf3 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 @@ -34,4 +34,4 @@ 406 406 16 406 406 16 406 406 16 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 index 7c591d6146fd6..4e3bbb2779bf3 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 @@ -34,4 +34,4 @@ 406 406 16 406 406 16 406 406 16 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 index 7c591d6146fd6..4e3bbb2779bf3 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 @@ -34,4 +34,4 @@ 406 406 16 406 406 16 406 406 16 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 b/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 index 8338433cd5a27..917ab36da2fbb 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 @@ -34,4 +34,4 @@ 406 4 1 406 val_406 406 4 1 406 val_406 406 4 1 406 val_406 -406 4 1 406 val_406 \ No newline at end of file +406 4 1 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 b/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 index 8338433cd5a27..917ab36da2fbb 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 @@ -34,4 +34,4 @@ 406 4 1 406 val_406 406 4 1 406 val_406 406 4 1 406 val_406 -406 4 1 406 val_406 \ No newline at end of file +406 4 1 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d index 8b1d3cd388fa2..a6942feae66e5 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d @@ -12,4 +12,4 @@ 311 1 311 val_311 9 369 1 369 val_369 9 401 1 401 val_401 25 -406 1 406 val_406 16 \ No newline at end of file +406 1 406 val_406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d index 8b1d3cd388fa2..a6942feae66e5 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d @@ -12,4 +12,4 @@ 311 1 311 val_311 9 369 1 369 val_369 9 401 1 401 val_401 25 -406 1 406 val_406 16 \ No newline at end of file +406 1 406 val_406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d index 8b1d3cd388fa2..a6942feae66e5 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d @@ -12,4 +12,4 @@ 311 1 311 val_311 9 369 1 369 val_369 9 401 1 401 val_401 25 -406 1 406 val_406 16 \ No newline at end of file +406 1 406 val_406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 index 19b8a2aea8f64..8f9dae31cc51c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 @@ -12,4 +12,4 @@ 311 1 311 9 369 1 369 9 401 1 401 25 -406 1 406 16 \ No newline at end of file +406 1 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 index 747b650237b53..1a82dfc153565 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 @@ -12,4 +12,4 @@ 311 3 311 val_311 369 3 369 401 5 401 val_401 -406 4 406 val_406 \ No newline at end of file +406 4 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 index 747b650237b53..1a82dfc153565 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 @@ -12,4 +12,4 @@ 311 3 311 val_311 369 3 369 401 5 401 val_401 -406 4 406 val_406 \ No newline at end of file +406 4 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 index 747b650237b53..1a82dfc153565 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 @@ -12,4 +12,4 @@ 311 3 311 val_311 369 3 369 401 5 401 val_401 -406 4 406 val_406 \ No newline at end of file +406 4 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 index 747b650237b53..1a82dfc153565 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 @@ -12,4 +12,4 @@ 311 3 311 val_311 369 3 369 401 5 401 val_401 -406 4 406 val_406 \ No newline at end of file +406 4 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d b/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f b/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f index 7c0af7229d62a..ecbe52c536ebe 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f @@ -21,4 +21,4 @@ val_273 1 NULL NULL val_278 1 NULL NULL val_311 1 NULL NULL val_401 1 NULL NULL -val_406 1 NULL NULL \ No newline at end of file +val_406 1 NULL NULL diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 b/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 index efdd80c9f8b89..2f62508e3342a 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 @@ -10,4 +10,4 @@ 311 val_311 3 369 3 401 val_401 5 -406 val_406 4 \ No newline at end of file +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 b/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 index efdd80c9f8b89..2f62508e3342a 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 @@ -10,4 +10,4 @@ 311 val_311 3 369 3 401 val_401 5 -406 val_406 4 \ No newline at end of file +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f b/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f index 7c0af7229d62a..ecbe52c536ebe 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f @@ -21,4 +21,4 @@ val_273 1 NULL NULL val_278 1 NULL NULL val_311 1 NULL NULL val_401 1 NULL NULL -val_406 1 NULL NULL \ No newline at end of file +val_406 1 NULL NULL diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 b/sql/hive/src/test/resources/golden/count-1-c47c4abedf055b4e734232fd2e274d55 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 rename to sql/hive/src/test/resources/golden/count-1-c47c4abedf055b4e734232fd2e274d55 diff --git a/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 b/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 index 4b020e0595d2c..966f27f6c9b9b 100644 --- a/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 +++ b/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 @@ -1,4 +1,4 @@ NULL 1 1 6 10 2 2 10 12 1 2 9 -100 1 1 3 \ No newline at end of file +100 1 1 3 diff --git a/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 b/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 index 54199fdb8166d..5eec149bfcc92 100644 --- a/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 +++ b/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 @@ -1 +1 @@ -7 7 6 6 6 7 3 3 6 7 4 5 6 6 5 6 4 5 5 5 4 \ No newline at end of file +7 7 6 6 6 7 3 3 6 7 4 5 6 6 5 6 4 5 5 5 4 diff --git a/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 b/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 index 2e9278da88858..162877fdb3e5d 100644 --- a/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 +++ b/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 @@ -4,4 +4,4 @@ NULL 35 23 6 12 NULL 80 2 10 100 NULL 5 10 100 45 4 -12 100 75 7 \ No newline at end of file +12 100 75 7 diff --git a/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 b/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 index 4b020e0595d2c..966f27f6c9b9b 100644 --- a/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 +++ b/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 @@ -1,4 +1,4 @@ NULL 1 1 6 10 2 2 10 12 1 2 9 -100 1 1 3 \ No newline at end of file +100 1 1 3 diff --git a/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 b/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 index 54199fdb8166d..5eec149bfcc92 100644 --- a/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 +++ b/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 @@ -1 +1 @@ -7 7 6 6 6 7 3 3 6 7 4 5 6 6 5 6 4 5 5 5 4 \ No newline at end of file +7 7 6 6 6 7 3 3 6 7 4 5 6 6 5 6 4 5 5 5 4 diff --git a/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f b/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f deleted file mode 100644 index 7aae61e5eb82f..0000000000000 --- a/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f +++ /dev/null @@ -1,500 +0,0 @@ -238 val_238 -86 val_86 -311 val_311 -27 val_27 -165 val_165 -409 val_409 -255 val_255 -278 val_278 -98 val_98 -484 val_484 -265 val_265 -193 val_193 -401 val_401 -150 val_150 -273 val_273 -224 val_224 -369 val_369 -66 val_66 -128 val_128 -213 val_213 -146 val_146 -406 val_406 -429 val_429 -374 val_374 -152 val_152 -469 val_469 -145 val_145 -495 val_495 -37 val_37 -327 val_327 -281 val_281 -277 val_277 -209 val_209 -15 val_15 -82 val_82 -403 val_403 -166 val_166 -417 val_417 -430 val_430 -252 val_252 -292 val_292 -219 val_219 -287 val_287 -153 val_153 -193 val_193 -338 val_338 -446 val_446 -459 val_459 -394 val_394 -237 val_237 -482 val_482 -174 val_174 -413 val_413 -494 val_494 -207 val_207 -199 val_199 -466 val_466 -208 val_208 -174 val_174 -399 val_399 -396 val_396 -247 val_247 -417 val_417 -489 val_489 -162 val_162 -377 val_377 -397 val_397 -309 val_309 -365 val_365 -266 val_266 -439 val_439 -342 val_342 -367 val_367 -325 val_325 -167 val_167 -195 val_195 -475 val_475 -17 val_17 -113 val_113 -155 val_155 -203 val_203 -339 val_339 -0 val_0 -455 val_455 -128 val_128 -311 val_311 -316 val_316 -57 val_57 -302 val_302 -205 val_205 -149 val_149 -438 val_438 -345 val_345 -129 val_129 -170 val_170 -20 val_20 -489 val_489 -157 val_157 -378 val_378 -221 val_221 -92 val_92 -111 val_111 -47 val_47 -72 val_72 -4 val_4 -280 val_280 -35 val_35 -427 val_427 -277 val_277 -208 val_208 -356 val_356 -399 val_399 -169 val_169 -382 val_382 -498 val_498 -125 val_125 -386 val_386 -437 val_437 -469 val_469 -192 val_192 -286 val_286 -187 val_187 -176 val_176 -54 val_54 -459 val_459 -51 val_51 -138 val_138 -103 val_103 -239 val_239 -213 val_213 -216 val_216 -430 val_430 -278 val_278 -176 val_176 -289 val_289 -221 val_221 -65 val_65 -318 val_318 -332 val_332 -311 val_311 -275 val_275 -137 val_137 -241 val_241 -83 val_83 -333 val_333 -180 val_180 -284 val_284 -12 val_12 -230 val_230 -181 val_181 -67 val_67 -260 val_260 -404 val_404 -384 val_384 -489 val_489 -353 val_353 -373 val_373 -272 val_272 -138 val_138 -217 val_217 -84 val_84 -348 val_348 -466 val_466 -58 val_58 -8 val_8 -411 val_411 -230 val_230 -208 val_208 -348 val_348 -24 val_24 -463 val_463 -431 val_431 -179 val_179 -172 val_172 -42 val_42 -129 val_129 -158 val_158 -119 val_119 -496 val_496 -0 val_0 -322 val_322 -197 val_197 -468 val_468 -393 val_393 -454 val_454 -100 val_100 -298 val_298 -199 val_199 -191 val_191 -418 val_418 -96 val_96 -26 val_26 -165 val_165 -327 val_327 -230 val_230 -205 val_205 -120 val_120 -131 val_131 -51 val_51 -404 val_404 -43 val_43 -436 val_436 -156 val_156 -469 val_469 -468 val_468 -308 val_308 -95 val_95 -196 val_196 -288 val_288 -481 val_481 -457 val_457 -98 val_98 -282 val_282 -197 val_197 -187 val_187 -318 val_318 -318 val_318 -409 val_409 -470 val_470 -137 val_137 -369 val_369 -316 val_316 -169 val_169 -413 val_413 -85 val_85 -77 val_77 -0 val_0 -490 val_490 -87 val_87 -364 val_364 -179 val_179 -118 val_118 -134 val_134 -395 val_395 -282 val_282 -138 val_138 -238 val_238 -419 val_419 -15 val_15 -118 val_118 -72 val_72 -90 val_90 -307 val_307 -19 val_19 -435 val_435 -10 val_10 -277 val_277 -273 val_273 -306 val_306 -224 val_224 -309 val_309 -389 val_389 -327 val_327 -242 val_242 -369 val_369 -392 val_392 -272 val_272 -331 val_331 -401 val_401 -242 val_242 -452 val_452 -177 val_177 -226 val_226 -5 val_5 -497 val_497 -402 val_402 -396 val_396 -317 val_317 -395 val_395 -58 val_58 -35 val_35 -336 val_336 -95 val_95 -11 val_11 -168 val_168 -34 val_34 -229 val_229 -233 val_233 -143 val_143 -472 val_472 -322 val_322 -498 val_498 -160 val_160 -195 val_195 -42 val_42 -321 val_321 -430 val_430 -119 val_119 -489 val_489 -458 val_458 -78 val_78 -76 val_76 -41 val_41 -223 val_223 -492 val_492 -149 val_149 -449 val_449 -218 val_218 -228 val_228 -138 val_138 -453 val_453 -30 val_30 -209 val_209 -64 val_64 -468 val_468 -76 val_76 -74 val_74 -342 val_342 -69 val_69 -230 val_230 -33 val_33 -368 val_368 -103 val_103 -296 val_296 -113 val_113 -216 val_216 -367 val_367 -344 val_344 -167 val_167 -274 val_274 -219 val_219 -239 val_239 -485 val_485 -116 val_116 -223 val_223 -256 val_256 -263 val_263 -70 val_70 -487 val_487 -480 val_480 -401 val_401 -288 val_288 -191 val_191 -5 val_5 -244 val_244 -438 val_438 -128 val_128 -467 val_467 -432 val_432 -202 val_202 -316 val_316 -229 val_229 -469 val_469 -463 val_463 -280 val_280 -2 val_2 -35 val_35 -283 val_283 -331 val_331 -235 val_235 -80 val_80 -44 val_44 -193 val_193 -321 val_321 -335 val_335 -104 val_104 -466 val_466 -366 val_366 -175 val_175 -403 val_403 -483 val_483 -53 val_53 -105 val_105 -257 val_257 -406 val_406 -409 val_409 -190 val_190 -406 val_406 -401 val_401 -114 val_114 -258 val_258 -90 val_90 -203 val_203 -262 val_262 -348 val_348 -424 val_424 -12 val_12 -396 val_396 -201 val_201 -217 val_217 -164 val_164 -431 val_431 -454 val_454 -478 val_478 -298 val_298 -125 val_125 -431 val_431 -164 val_164 -424 val_424 -187 val_187 -382 val_382 -5 val_5 -70 val_70 -397 val_397 -480 val_480 -291 val_291 -24 val_24 -351 val_351 -255 val_255 -104 val_104 -70 val_70 -163 val_163 -438 val_438 -119 val_119 -414 val_414 -200 val_200 -491 val_491 -237 val_237 -439 val_439 -360 val_360 -248 val_248 -479 val_479 -305 val_305 -417 val_417 -199 val_199 -444 val_444 -120 val_120 -429 val_429 -169 val_169 -443 val_443 -323 val_323 -325 val_325 -277 val_277 -230 val_230 -478 val_478 -178 val_178 -468 val_468 -310 val_310 -317 val_317 -333 val_333 -493 val_493 -460 val_460 -207 val_207 -249 val_249 -265 val_265 -480 val_480 -83 val_83 -136 val_136 -353 val_353 -172 val_172 -214 val_214 -462 val_462 -233 val_233 -406 val_406 -133 val_133 -175 val_175 -189 val_189 -454 val_454 -375 val_375 -401 val_401 -421 val_421 -407 val_407 -384 val_384 -256 val_256 -26 val_26 -134 val_134 -67 val_67 -384 val_384 -379 val_379 -18 val_18 -462 val_462 -492 val_492 -100 val_100 -298 val_298 -9 val_9 -341 val_341 -498 val_498 -146 val_146 -458 val_458 -362 val_362 -186 val_186 -285 val_285 -348 val_348 -167 val_167 -18 val_18 -273 val_273 -183 val_183 -281 val_281 -344 val_344 -97 val_97 -469 val_469 -315 val_315 -84 val_84 -28 val_28 -37 val_37 -448 val_448 -152 val_152 -348 val_348 -307 val_307 -194 val_194 -414 val_414 -477 val_477 -222 val_222 -126 val_126 -90 val_90 -169 val_169 -403 val_403 -400 val_400 -200 val_200 -97 val_97 diff --git a/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 b/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 +++ b/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba b/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba deleted file mode 100644 index 66d40e52a4539..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b string None - -Detailed Table Information Table(tableName:table3, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 b/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 deleted file mode 100644 index 8e606f61a1c30..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a b/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a deleted file mode 100644 index 05b460270525d..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b string None - -Detailed Table Information Table(tableName:table4, dbName:default, owner:marmbrus, createTime:1389343869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table4, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 b/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 deleted file mode 100644 index eda3946588e3f..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 +++ /dev/null @@ -1,2 +0,0 @@ -a string from deserializer -b string from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 b/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 deleted file mode 100644 index 8fc60adf10167..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 +++ /dev/null @@ -1,4 +0,0 @@ -a string from deserializer -b string from deserializer - -Detailed Table Information Table(tableName:table5, dbName:default, owner:marmbrus, createTime:1389343869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table5, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 deleted file mode 100644 index 8e606f61a1c30..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 deleted file mode 100644 index 5e9bc70c3a1a3..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b string None - -Detailed Table Information Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389343868}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a deleted file mode 100644 index 6ed3515ba6876..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b int None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 b/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 deleted file mode 100644 index bec4c72e49857..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b int None - -Detailed Table Information Table(tableName:table2, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389343868}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 deleted file mode 100644 index 8e606f61a1c30..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 deleted file mode 100644 index 8e606f61a1c30..0000000000000 --- a/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 deleted file mode 100644 index 8ab6b24deab60..0000000000000 --- a/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b string None - -Detailed Table Information Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1388825524, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3286459698772672096/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{escape.delim=\, serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 b/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 deleted file mode 100644 index bbe37f8e2a790..0000000000000 --- a/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 +++ /dev/null @@ -1 +0,0 @@ -86 \ \ \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 b/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 deleted file mode 100644 index cbcdfbe72e8c6..0000000000000 --- a/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 +++ /dev/null @@ -1,9 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_avg 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 b/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 deleted file mode 100644 index 2f958c483a9df..0000000000000 --- a/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 +++ /dev/null @@ -1 +0,0 @@ -1.0 260.182 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 b/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 index 0e9c06c882602..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 +++ b/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_translate 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestTranslate') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc index 83f572c424926..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc +++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc @@ -1,2 +1,2 @@ -key int None -value string None \ No newline at end of file +key int +value string diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c index 83f572c424926..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c +++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c @@ -1,2 +1,2 @@ -key int None -value string None \ No newline at end of file +key int +value string diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 index 83f572c424926..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 +++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 @@ -1,2 +1,2 @@ -key int None -value string None \ No newline at end of file +key int +value string diff --git a/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af deleted file mode 100644 index b8ddba7f50b97..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -a string None -b string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 11:13:23 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392059603 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 b/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 deleted file mode 100644 index e87fb81a6f043..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 +++ /dev/null @@ -1 +0,0 @@ -86 val_86 diff --git a/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 b/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 deleted file mode 100644 index 375dcacb8db71..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 +++ /dev/null @@ -1,2 +0,0 @@ -100 val_100 -100 val_100 diff --git a/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 deleted file mode 100644 index 52b637c27f98d..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -a string None -b string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 11:13:23 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table2 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392059603 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 deleted file mode 100644 index d9308798dcb1c..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 +++ /dev/null @@ -1,28 +0,0 @@ -# col_name data_type comment - -a string None -b string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 11:13:23 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table3 -Table Type: EXTERNAL_TABLE -Table Parameters: - EXTERNAL TRUE - transient_lastDdlTime 1392059603 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 index 3e2a912824ab3..0e52b5f82ef4a 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 +++ b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 @@ -1,19 +1,19 @@ # col_name data_type comment -a string None -b string None +a string +b string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:52:37 PST 2014 +CreateTime: Tue Oct 21 01:24:36 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table2 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table2 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1391813557 + transient_lastDdlTime 1413879876 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 index 115d12fb70c81..26b5989488752 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 +++ b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 @@ -1,2 +1,2 @@ -a string None -b string None +a string +b string diff --git a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 index 61d34badb1a2d..47808e8f20c83 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 +++ b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 @@ -1,20 +1,20 @@ # col_name data_type comment -a string None -b string None +a string +b string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:52:37 PST 2014 +CreateTime: Tue Oct 21 01:24:36 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table3 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table3 Table Type: EXTERNAL_TABLE Table Parameters: EXTERNAL TRUE - transient_lastDdlTime 1391813557 + transient_lastDdlTime 1413879876 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d index 4c2967215fe66..ad1f1f42d6b98 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d +++ b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d @@ -1,25 +1,25 @@ # col_name data_type comment -key int None -value string None +key int +value string # Partition Information # col_name data_type comment -ds string None -hr string None +ds string +hr string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:53:16 PST 2014 +CreateTime: Tue Oct 21 01:25:10 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1391813596 + transient_lastDdlTime 1413879910 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 index 115d12fb70c81..26b5989488752 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 +++ b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 @@ -1,2 +1,2 @@ -a string None -b string None +a string +b string diff --git a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af index 02dee147bca42..91e1ebbfee4de 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af +++ b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af @@ -1,19 +1,19 @@ # col_name data_type comment -a string None -b string None +a string +b string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:52:37 PST 2014 +CreateTime: Tue Oct 21 01:24:36 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1391813557 + transient_lastDdlTime 1413879876 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a index 115d12fb70c81..26b5989488752 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a +++ b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a @@ -1,2 +1,2 @@ -a string None -b string None +a string +b string diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 b/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 deleted file mode 100644 index dbdf4585360ae..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:tgt_rc_merge_test -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/tgt_rc_merge_test -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:239 -maxFileSize:239 -minFileSize:239 -lastAccessTime:0 -lastUpdateTime:1389344017000 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d b/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d deleted file mode 100644 index 9a037142aa3c1..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d +++ /dev/null @@ -1 +0,0 @@ -10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 b/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 deleted file mode 100644 index eb6c3f6aef813..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 +++ /dev/null @@ -1 +0,0 @@ -508 -751895388 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 b/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 deleted file mode 100644 index 8a9d9d56a66d6..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:tgt_rc_merge_test -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/tgt_rc_merge_test -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:2 -totalFileSize:338 -maxFileSize:169 -minFileSize:169 -lastAccessTime:0 -lastUpdateTime:1389343990000 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d b/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d deleted file mode 100644 index 9a037142aa3c1..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d +++ /dev/null @@ -1 +0,0 @@ -10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 b/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 deleted file mode 100644 index eb6c3f6aef813..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 +++ /dev/null @@ -1 +0,0 @@ -508 -751895388 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 index c49ad26c04d67..6485e594f54fe 100644 --- a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 +++ b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 @@ -1,4 +1,4 @@ -a string None -b array None -c array> None -d map> None +a string +b array +c array> +d map> diff --git a/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 index 41c7202c8b886..6d68db6387495 100644 --- a/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 +++ b/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 @@ -1,6 +1,6 @@ -a string None -b array None -c array> None -d map> None +a string +b array +c array> +d map> -Detailed Table Information Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1391226109, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:array, comment:null), FieldSchema(name:c, type:array>, comment:null), FieldSchema(name:d, type:map>, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1391226109}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1413879912, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:array, comment:null), FieldSchema(name:c, type:array>, comment:null), FieldSchema(name:d, type:map>, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413879912}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 b/sql/hive/src/test/resources/golden/create_nested_type-3-280fd0d9876e475d7dcf5383876ebc79 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 rename to sql/hive/src/test/resources/golden/create_nested_type-3-280fd0d9876e475d7dcf5383876ebc79 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 b/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 deleted file mode 100644 index 46869fc9e88eb..0000000000000 --- a/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 +++ /dev/null @@ -1,30 +0,0 @@ -# col_name data_type comment - -key int None -value string None -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 12:09:28 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Table Type: VIRTUAL_VIEW -Table Parameters: - transient_lastDdlTime 1392062968 - -# Storage Information -SerDe Library: null -InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] - -# View Information -View Original Text: select * from srcpart -View Expanded Text: select `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` from `default`.`srcpart` diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 b/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 index d7c386e17c096..d35fc35a810d3 100644 --- a/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 @@ -1,19 +1,19 @@ # col_name data_type comment -key string None -value string None +key string +value string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Tue Jan 14 22:51:44 PST 2014 +CreateTime: Tue Oct 21 01:25:13 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_single_2 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_single_2 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389768704 + transient_lastDdlTime 1413879913 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -26,4 +26,4 @@ Sort Columns: [] Skewed Columns: [key] Skewed Values: [[1], [5], [6]] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e b/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e index 0817efac83d79..e8f72845c1c7a 100644 --- a/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e @@ -1,19 +1,19 @@ # col_name data_type comment -key string None -value string None +key string +value string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Tue Jan 14 22:51:44 PST 2014 +CreateTime: Tue Oct 21 01:25:13 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_single +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_single Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389768704 + transient_lastDdlTime 1413879913 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -26,4 +26,4 @@ Sort Columns: [] Skewed Columns: [key] Skewed Values: [[1], [5], [6]] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef b/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef index 2986dd43b0fd6..c8d58bbb1b1ce 100644 --- a/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef @@ -1,20 +1,20 @@ # col_name data_type comment -col1 string None -col2 int None -col3 string None +col1 string +col2 int +col3 string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Tue Jan 14 22:51:44 PST 2014 +CreateTime: Tue Oct 21 01:25:13 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_multiple +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_multiple Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389768704 + transient_lastDdlTime 1413879913 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -27,4 +27,4 @@ Sort Columns: [] Skewed Columns: [col1, col2] Skewed Values: [[s1, 1], [s3, 3], [s13, 13], [s78, 78]] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 b/sql/hive/src/test/resources/golden/create_struct_table-1-719851d0e8b89b51bdc6be4177455a92 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 rename to sql/hive/src/test/resources/golden/create_struct_table-1-719851d0e8b89b51bdc6be4177455a92 diff --git a/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e b/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e deleted file mode 100644 index 8af6a0338d65a..0000000000000 --- a/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_max 'org.apache.hadoop.hive.ql.udf.UDAFTestMax') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b b/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b deleted file mode 100644 index e87fb81a6f043..0000000000000 --- a/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b +++ /dev/null @@ -1 +0,0 @@ -86 val_86 diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 b/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 deleted file mode 100644 index 70d2ecdbc8f78..0000000000000 --- a/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 +++ /dev/null @@ -1,13 +0,0 @@ -key int None -value string None - -# Partition Information -# col_name data_type comment - -value string None - -Detailed Table Information Table(tableName:vp1, dbName:default, owner:marmbrus, createTime:1392062982, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{transient_lastDdlTime=1392062982}, viewOriginalText:SELECT key, value -FROM src -WHERE key=86, viewExpandedText:SELECT `src`.`key`, `src`.`value` -FROM `default`.`src` -WHERE `src`.`key`=86, tableType:VIRTUAL_VIEW) diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc b/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc deleted file mode 100644 index 43bc3de44f4df..0000000000000 --- a/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None - -# Partition Information -# col_name data_type comment - -value string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 12:09:42 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Table Type: VIRTUAL_VIEW -Table Parameters: - transient_lastDdlTime 1392062982 - -# Storage Information -SerDe Library: null -InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] - -# View Information -View Original Text: SELECT key, value -FROM src -WHERE key=86 -View Expanded Text: SELECT `src`.`key`, `src`.`value` -FROM `default`.`src` -WHERE `src`.`key`=86 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 b/sql/hive/src/test/resources/golden/cross_product_check_1-0-d782db598869f9b19e0fcf5ea2a83594 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 rename to sql/hive/src/test/resources/golden/cross_product_check_1-0-d782db598869f9b19e0fcf5ea2a83594 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 b/sql/hive/src/test/resources/golden/cross_product_check_1-1-bd8395c55fa2fc80b68eb043b7020b76 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 rename to sql/hive/src/test/resources/golden/cross_product_check_1-1-bd8395c55fa2fc80b68eb043b7020b76 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/cross_product_check_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/cross_product_check_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 b/sql/hive/src/test/resources/golden/cross_product_check_1-3-32a3e6eb858d37f58e225d07fb323254 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 rename to sql/hive/src/test/resources/golden/cross_product_check_1-3-32a3e6eb858d37f58e225d07fb323254 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 b/sql/hive/src/test/resources/golden/cross_product_check_1-4-36a6b6fb71ea08ff817dd40d1ffdb970 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 rename to sql/hive/src/test/resources/golden/cross_product_check_1-4-36a6b6fb71ea08ff817dd40d1ffdb970 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 b/sql/hive/src/test/resources/golden/cross_product_check_1-5-103739f072b849d212dbc40919f92b74 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 rename to sql/hive/src/test/resources/golden/cross_product_check_1-5-103739f072b849d212dbc40919f92b74 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/cross_product_check_1-6-9a202e3bf15be2a310504a50920e7d25 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/cross_product_check_1-6-9a202e3bf15be2a310504a50920e7d25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/cross_product_check_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 rename to sql/hive/src/test/resources/golden/cross_product_check_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 b/sql/hive/src/test/resources/golden/cross_product_check_2-0-d782db598869f9b19e0fcf5ea2a83594 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 rename to sql/hive/src/test/resources/golden/cross_product_check_2-0-d782db598869f9b19e0fcf5ea2a83594 diff --git a/sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 b/sql/hive/src/test/resources/golden/cross_product_check_2-1-bd8395c55fa2fc80b68eb043b7020b76 similarity index 100% rename from sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 rename to sql/hive/src/test/resources/golden/cross_product_check_2-1-bd8395c55fa2fc80b68eb043b7020b76 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/cross_product_check_2-2-24ca942f094b14b92086305cc125e833 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/cross_product_check_2-2-24ca942f094b14b92086305cc125e833 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/cross_product_check_2-3-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/cross_product_check_2-3-44d382ce6848d3f0b900b0808747d8e9 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/cross_product_check_2-4-c14792ccac2ca64e3e9e21af4fd12d2c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/cross_product_check_2-4-c14792ccac2ca64e3e9e21af4fd12d2c diff --git a/sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 b/sql/hive/src/test/resources/golden/cross_product_check_2-5-32a3e6eb858d37f58e225d07fb323254 similarity index 100% rename from sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 rename to sql/hive/src/test/resources/golden/cross_product_check_2-5-32a3e6eb858d37f58e225d07fb323254 diff --git a/sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/cross_product_check_2-6-36a6b6fb71ea08ff817dd40d1ffdb970 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 rename to sql/hive/src/test/resources/golden/cross_product_check_2-6-36a6b6fb71ea08ff817dd40d1ffdb970 diff --git a/sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 b/sql/hive/src/test/resources/golden/cross_product_check_2-7-103739f072b849d212dbc40919f92b74 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 rename to sql/hive/src/test/resources/golden/cross_product_check_2-7-103739f072b849d212dbc40919f92b74 diff --git a/sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379 b/sql/hive/src/test/resources/golden/cross_product_check_2-8-9a202e3bf15be2a310504a50920e7d25 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379 rename to sql/hive/src/test/resources/golden/cross_product_check_2-8-9a202e3bf15be2a310504a50920e7d25 diff --git a/sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c b/sql/hive/src/test/resources/golden/cross_product_check_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c rename to sql/hive/src/test/resources/golden/cross_product_check_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec b/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec deleted file mode 100644 index 6839c16243bcd..0000000000000 --- a/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 b/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 deleted file mode 100644 index 6839c16243bcd..0000000000000 --- a/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 diff --git a/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd b/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd +++ b/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e b/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e +++ b/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e b/sql/hive/src/test/resources/golden/database_drop-10-8db536f925bf0f5058f97897e145a661 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e rename to sql/hive/src/test/resources/golden/database_drop-10-8db536f925bf0f5058f97897e145a661 diff --git a/sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 b/sql/hive/src/test/resources/golden/database_drop-11-1b0a6cff3151cfa0ef0a6f78587973a5 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 rename to sql/hive/src/test/resources/golden/database_drop-11-1b0a6cff3151cfa0ef0a6f78587973a5 diff --git a/sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 b/sql/hive/src/test/resources/golden/database_drop-12-2ea883422b74b701711e14e61472ba06 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 rename to sql/hive/src/test/resources/golden/database_drop-12-2ea883422b74b701711e14e61472ba06 diff --git a/sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 b/sql/hive/src/test/resources/golden/database_drop-13-e02a53f7e798d2741152526516f14941 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 rename to sql/hive/src/test/resources/golden/database_drop-13-e02a53f7e798d2741152526516f14941 diff --git a/sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d b/sql/hive/src/test/resources/golden/database_drop-14-8f0fe60664d020b2a42c7f5c0c7bed35 similarity index 100% rename from sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d rename to sql/hive/src/test/resources/golden/database_drop-14-8f0fe60664d020b2a42c7f5c0c7bed35 diff --git a/sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee b/sql/hive/src/test/resources/golden/database_drop-15-7928ac876f76c990fa21d74b6c9e14f6 similarity index 100% rename from sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee rename to sql/hive/src/test/resources/golden/database_drop-15-7928ac876f76c990fa21d74b6c9e14f6 diff --git a/sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e b/sql/hive/src/test/resources/golden/database_drop-16-f9036ff3f0a3101373bdbc9a52faf00e similarity index 100% rename from sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e rename to sql/hive/src/test/resources/golden/database_drop-16-f9036ff3f0a3101373bdbc9a52faf00e diff --git a/sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 b/sql/hive/src/test/resources/golden/database_drop-17-95b906fd73935da8746b5277170e91e8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 rename to sql/hive/src/test/resources/golden/database_drop-17-95b906fd73935da8746b5277170e91e8 diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b b/sql/hive/src/test/resources/golden/database_drop-18-f65bf675b83871af7906741a60fa1318 similarity index 100% rename from sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b rename to sql/hive/src/test/resources/golden/database_drop-18-f65bf675b83871af7906741a60fa1318 diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 b/sql/hive/src/test/resources/golden/database_drop-19-d7fefbf585dbb67491e871ef58dca752 similarity index 100% rename from sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 rename to sql/hive/src/test/resources/golden/database_drop-19-d7fefbf585dbb67491e871ef58dca752 diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 b/sql/hive/src/test/resources/golden/database_drop-20-cbb84f0db4d55930a85cff28f7400bd0 similarity index 100% rename from sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 rename to sql/hive/src/test/resources/golden/database_drop-20-cbb84f0db4d55930a85cff28f7400bd0 diff --git a/sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/database_drop-21-eea49f066631be60519ae9d6b614d7d0 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 rename to sql/hive/src/test/resources/golden/database_drop-21-eea49f066631be60519ae9d6b614d7d0 diff --git a/sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a b/sql/hive/src/test/resources/golden/database_drop-22-85833c3a68c29098827e438ff580bb94 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a rename to sql/hive/src/test/resources/golden/database_drop-22-85833c3a68c29098827e438ff580bb94 diff --git a/sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 b/sql/hive/src/test/resources/golden/database_drop-23-84a5672989118a1b5792474c1469de90 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 rename to sql/hive/src/test/resources/golden/database_drop-23-84a5672989118a1b5792474c1469de90 diff --git a/sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d b/sql/hive/src/test/resources/golden/database_drop-24-441116797e8d95554c74472fa7644440 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d rename to sql/hive/src/test/resources/golden/database_drop-24-441116797e8d95554c74472fa7644440 diff --git a/sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 b/sql/hive/src/test/resources/golden/database_drop-25-847ca97dd211889d83e5d487bfc44e23 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 rename to sql/hive/src/test/resources/golden/database_drop-25-847ca97dd211889d83e5d487bfc44e23 diff --git a/sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 b/sql/hive/src/test/resources/golden/database_drop-26-bea20178515df24fcca04c0384f1c1b7 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 rename to sql/hive/src/test/resources/golden/database_drop-26-bea20178515df24fcca04c0384f1c1b7 diff --git a/sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 b/sql/hive/src/test/resources/golden/database_drop-27-cb4b90a7f63c1646319ee7bb014a7750 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 rename to sql/hive/src/test/resources/golden/database_drop-27-cb4b90a7f63c1646319ee7bb014a7750 diff --git a/sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb b/sql/hive/src/test/resources/golden/database_drop-28-53d67cbed634cff012dac41340bf6630 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb rename to sql/hive/src/test/resources/golden/database_drop-28-53d67cbed634cff012dac41340bf6630 diff --git a/sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb b/sql/hive/src/test/resources/golden/database_drop-29-29d3232325eda40cbb0bd1786b7d070e similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb rename to sql/hive/src/test/resources/golden/database_drop-29-29d3232325eda40cbb0bd1786b7d070e diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/database_drop-3-db64b724719d27c7f0db4f51f5c4edaa similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/database_drop-3-db64b724719d27c7f0db4f51f5c4edaa diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c b/sql/hive/src/test/resources/golden/database_drop-30-bbad0860316d8b9b1eed50d231f3ab5d similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c rename to sql/hive/src/test/resources/golden/database_drop-30-bbad0860316d8b9b1eed50d231f3ab5d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 b/sql/hive/src/test/resources/golden/database_drop-31-981f8e58caeae9cbbad3a113e3043de5 similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 rename to sql/hive/src/test/resources/golden/database_drop-31-981f8e58caeae9cbbad3a113e3043de5 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f b/sql/hive/src/test/resources/golden/database_drop-32-6c8e7590238b5aca1772721f0b914ece similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f rename to sql/hive/src/test/resources/golden/database_drop-32-6c8e7590238b5aca1772721f0b914ece diff --git a/sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 b/sql/hive/src/test/resources/golden/database_drop-33-2bc7864932f597bdf98bdc410b1a2d9c similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 rename to sql/hive/src/test/resources/golden/database_drop-33-2bc7864932f597bdf98bdc410b1a2d9c diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/database_drop-34-491138bed44a70cb783bb2b531e1d82 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 rename to sql/hive/src/test/resources/golden/database_drop-34-491138bed44a70cb783bb2b531e1d82 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/database_drop-35-9e0285d0596607cdadf75a763a543866 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 rename to sql/hive/src/test/resources/golden/database_drop-35-9e0285d0596607cdadf75a763a543866 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 b/sql/hive/src/test/resources/golden/database_drop-36-e66471f3f1bbe2d4b3b214e47793656d similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 rename to sql/hive/src/test/resources/golden/database_drop-36-e66471f3f1bbe2d4b3b214e47793656d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 b/sql/hive/src/test/resources/golden/database_drop-37-f6410721e3125a89836817136306eac4 similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 rename to sql/hive/src/test/resources/golden/database_drop-37-f6410721e3125a89836817136306eac4 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 b/sql/hive/src/test/resources/golden/database_drop-38-7d45d97adebe50917a94cbe232c112a8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 rename to sql/hive/src/test/resources/golden/database_drop-38-7d45d97adebe50917a94cbe232c112a8 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 b/sql/hive/src/test/resources/golden/database_drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1 similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 rename to sql/hive/src/test/resources/golden/database_drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/database_drop-4-f21dd011aeb417043ed58c03fd5c3bf0 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 rename to sql/hive/src/test/resources/golden/database_drop-4-f21dd011aeb417043ed58c03fd5c3bf0 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 b/sql/hive/src/test/resources/golden/database_drop-40-10073fb74a5c792322fc52389997695b similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 rename to sql/hive/src/test/resources/golden/database_drop-40-10073fb74a5c792322fc52389997695b diff --git a/sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c b/sql/hive/src/test/resources/golden/database_drop-41-7164c585e1ef4d9036ed4db275811084 similarity index 100% rename from sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c rename to sql/hive/src/test/resources/golden/database_drop-41-7164c585e1ef4d9036ed4db275811084 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/database_drop-42-c55cffbfc4d950363be97ccdb028faf3 similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d rename to sql/hive/src/test/resources/golden/database_drop-42-c55cffbfc4d950363be97ccdb028faf3 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/database_drop-43-cc6860c125b8b62450cb858c72716dc2 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to sql/hive/src/test/resources/golden/database_drop-43-cc6860c125b8b62450cb858c72716dc2 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/database_drop-44-de81fd80132350aedcd9f0d9a212fd94 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/database_drop-44-de81fd80132350aedcd9f0d9a212fd94 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea b/sql/hive/src/test/resources/golden/database_drop-45-64e22634663e57153119340583e93651 similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea rename to sql/hive/src/test/resources/golden/database_drop-45-64e22634663e57153119340583e93651 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d b/sql/hive/src/test/resources/golden/database_drop-46-eb28b907b605e51026f9902287e1d90d similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d rename to sql/hive/src/test/resources/golden/database_drop-46-eb28b907b605e51026f9902287e1d90d diff --git a/sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/database_drop-47-44e4d2bdab2dfa4583da47281ed00ba3 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/database_drop-47-44e4d2bdab2dfa4583da47281ed00ba3 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/database_drop-48-d419f4ff197d4291208c2028cd158909 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/database_drop-48-d419f4ff197d4291208c2028cd158909 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/database_drop-5-d419f4ff197d4291208c2028cd158909 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to sql/hive/src/test/resources/golden/database_drop-5-d419f4ff197d4291208c2028cd158909 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/database_drop-6-b7cf74929eabe781b0db79ed1043dc24 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 rename to sql/hive/src/test/resources/golden/database_drop-6-b7cf74929eabe781b0db79ed1043dc24 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de b/sql/hive/src/test/resources/golden/database_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9 similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de rename to sql/hive/src/test/resources/golden/database_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb b/sql/hive/src/test/resources/golden/database_drop-8-b3980119a4b2d8f531951a940cba3697 similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb rename to sql/hive/src/test/resources/golden/database_drop-8-b3980119a4b2d8f531951a940cba3697 diff --git a/sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 b/sql/hive/src/test/resources/golden/database_drop-9-97101266791d2b2c662bcde549422318 similarity index 100% rename from sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 rename to sql/hive/src/test/resources/golden/database_drop-9-97101266791d2b2c662bcde549422318 diff --git a/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 b/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 index f05305df0c3d3..a97bfdfeef098 100644 --- a/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 +++ b/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 @@ -1 +1 @@ -db1 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db \ No newline at end of file +db1 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db1.db marmbrus diff --git a/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 b/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 index e92c241d50092..adf00309e0fae 100644 --- a/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 +++ b/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 @@ -1,19 +1,19 @@ # col_name data_type comment -name string None -value int None +name string +value int # Detailed Table Information Database: db2 Owner: marmbrus -CreateTime: Tue Jan 14 11:40:51 PST 2014 +CreateTime: Tue Oct 21 01:25:24 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2/table_db2 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2/table_db2 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389728451 + transient_lastDdlTime 1413879924 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c index 7c77c06cda82b..2e4bf94808285 100644 --- a/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c @@ -1 +1 @@ -table_db2 \ No newline at end of file +table_db2 diff --git a/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb b/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb index cda1736e9ca6b..e7c766a71c1d2 100644 --- a/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb +++ b/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb @@ -1,19 +1,19 @@ # col_name data_type comment -name string None -value int None +name string +value int # Detailed Table Information Database: db1 Owner: marmbrus -CreateTime: Tue Jan 14 11:40:50 PST 2014 +CreateTime: Tue Oct 21 01:25:24 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/table_db1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db1.db/table_db1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389728450 + transient_lastDdlTime 1413879924 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c index 4d8fc6a6d6fe0..6eabc06bb1362 100644 --- a/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c @@ -1 +1 @@ -table_db1 \ No newline at end of file +table_db1 diff --git a/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 b/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 index 3e23970adddcf..65f9d79a3c381 100644 --- a/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 +++ b/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 @@ -1 +1 @@ -db2 database 2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2 \ No newline at end of file +db2 database 2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2 marmbrus diff --git a/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ b/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 b/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 +++ b/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 b/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 index 2f5fbe26f4945..4c04259aed3a7 100644 --- a/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 +++ b/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 @@ -1 +1 @@ -db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db {mapred.jobtracker.url=http://my.jobtracker.com:53000, new.property=some new props, mapred.scratch.dir=hdfs://tmp.dfs.com:50029/tmp, hive.warehouse.dir=new/warehouse/dir} \ No newline at end of file +db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db marmbrus diff --git a/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 b/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 index 86362ead004d9..3cd776a0711ff 100644 --- a/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 +++ b/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 @@ -1,2 +1,2 @@ db1 -default \ No newline at end of file +default diff --git a/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a b/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a index ff89c3fe899d2..4c04259aed3a7 100644 --- a/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a +++ b/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a @@ -1 +1 @@ -db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db \ No newline at end of file +db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db marmbrus diff --git a/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 b/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 index 5827d2726d084..4c04259aed3a7 100644 --- a/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 +++ b/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 @@ -1 +1 @@ -db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db {mapred.jobtracker.url=http://my.jobtracker.com:53000, mapred.scratch.dir=hdfs://tmp.dfs.com:50029/tmp, hive.warehouse.dir=/user/hive/warehouse} \ No newline at end of file +db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db marmbrus diff --git a/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f +++ b/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 b/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 +++ b/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 b/sql/hive/src/test/resources/golden/date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 rename to sql/hive/src/test/resources/golden/date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1 diff --git a/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 b/sql/hive/src/test/resources/golden/date_3-3-c26f0641e7cec1093273b258e6bf7120 similarity index 100% rename from sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 rename to sql/hive/src/test/resources/golden/date_3-3-c26f0641e7cec1093273b258e6bf7120 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/date_4-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/date_4-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 b/sql/hive/src/test/resources/golden/date_4-1-b84f7e931d710dcbe3c5126d998285a8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 rename to sql/hive/src/test/resources/golden/date_4-1-b84f7e931d710dcbe3c5126d998285a8 diff --git a/sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/sql/hive/src/test/resources/golden/date_4-2-6272f5e518f6a20bc96a5870ff315c4f similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 rename to sql/hive/src/test/resources/golden/date_4-2-6272f5e518f6a20bc96a5870ff315c4f diff --git a/sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 b/sql/hive/src/test/resources/golden/date_4-3-4a0e7bde447ef616b98e0f55d2886de0 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 rename to sql/hive/src/test/resources/golden/date_4-3-4a0e7bde447ef616b98e0f55d2886de0 diff --git a/sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd b/sql/hive/src/test/resources/golden/date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd rename to sql/hive/src/test/resources/golden/date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8 diff --git a/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 b/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 new file mode 100644 index 0000000000000..b61affde4ffce --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 @@ -0,0 +1 @@ +2011-01-01 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea b/sql/hive/src/test/resources/golden/date_4-6-b84f7e931d710dcbe3c5126d998285a8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea rename to sql/hive/src/test/resources/golden/date_4-6-b84f7e931d710dcbe3c5126d998285a8 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/date_comparison-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/date_comparison-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 b/sql/hive/src/test/resources/golden/date_comparison-1-69eec445bd045c9dc899fafa348d8495 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 rename to sql/hive/src/test/resources/golden/date_comparison-1-69eec445bd045c9dc899fafa348d8495 diff --git a/sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 b/sql/hive/src/test/resources/golden/date_comparison-10-bcd987341fc1c38047a27d29dac6ae7c similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 rename to sql/hive/src/test/resources/golden/date_comparison-10-bcd987341fc1c38047a27d29dac6ae7c diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc b/sql/hive/src/test/resources/golden/date_comparison-11-a9f2560c273163e11306d4f1dd1d9d54 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc rename to sql/hive/src/test/resources/golden/date_comparison-11-a9f2560c273163e11306d4f1dd1d9d54 diff --git a/sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 b/sql/hive/src/test/resources/golden/date_comparison-12-4a7bac9ddcf40db6329faaec8e426543 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 rename to sql/hive/src/test/resources/golden/date_comparison-12-4a7bac9ddcf40db6329faaec8e426543 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 b/sql/hive/src/test/resources/golden/date_comparison-2-fcc400871a502009c8680509e3869ec1 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 rename to sql/hive/src/test/resources/golden/date_comparison-2-fcc400871a502009c8680509e3869ec1 diff --git a/sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 b/sql/hive/src/test/resources/golden/date_comparison-3-b8598a4d0c948c2ddcf3eeef0abf2264 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 rename to sql/hive/src/test/resources/golden/date_comparison-3-b8598a4d0c948c2ddcf3eeef0abf2264 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f b/sql/hive/src/test/resources/golden/date_comparison-4-14d35f266be9cceb11a2ae09ec8b3835 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f rename to sql/hive/src/test/resources/golden/date_comparison-4-14d35f266be9cceb11a2ae09ec8b3835 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-5-c8865b14d53f2c2496fb69ee8191bf37 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/date_comparison-5-c8865b14d53f2c2496fb69ee8191bf37 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-6-f2c907e64da8166a731ddc0ed19bad6c similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/date_comparison-6-f2c907e64da8166a731ddc0ed19bad6c diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 b/sql/hive/src/test/resources/golden/date_comparison-7-5606505a92bad10023ad9a3ef77eacc9 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 rename to sql/hive/src/test/resources/golden/date_comparison-7-5606505a92bad10023ad9a3ef77eacc9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-8-47913d4aaf0d468ab3764cc3bfd68eb similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/date_comparison-8-47913d4aaf0d468ab3764cc3bfd68eb diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b b/sql/hive/src/test/resources/golden/date_comparison-9-1e5ce4f833b6fba45618437c8fb7643c similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b rename to sql/hive/src/test/resources/golden/date_comparison-9-1e5ce4f833b6fba45618437c8fb7643c diff --git a/sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc b/sql/hive/src/test/resources/golden/date_join1-2-e967e1ef6b209dfa5bdc60021dcb1964 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc rename to sql/hive/src/test/resources/golden/date_join1-2-e967e1ef6b209dfa5bdc60021dcb1964 diff --git a/sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a b/sql/hive/src/test/resources/golden/date_serde-7-580096b3b48db26bea91b80e1e1b081a similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a rename to sql/hive/src/test/resources/golden/date_serde-7-580096b3b48db26bea91b80e1e1b081a diff --git a/sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 b/sql/hive/src/test/resources/golden/date_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 rename to sql/hive/src/test/resources/golden/date_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d diff --git a/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 b/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 deleted file mode 100644 index c270c7cbdfa1f..0000000000000 --- a/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 +++ /dev/null @@ -1,4 +0,0 @@ -key int None -value string None - -Detailed Table Information Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1392063041, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392063041}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 b/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 b/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 b/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 deleted file mode 100644 index c4a17c1b14c88..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 +++ /dev/null @@ -1 +0,0 @@ -1969-12-31 16:00:17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 b/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b b/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a b/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d b/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c b/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 b/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 b/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 b/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 deleted file mode 100644 index b1bd38b62a080..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 +++ /dev/null @@ -1 +0,0 @@ -13 diff --git a/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 b/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 deleted file mode 100644 index de7771ac23570..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 +++ /dev/null @@ -1 +0,0 @@ --3827 diff --git a/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 b/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 deleted file mode 100644 index 272791f402250..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 +++ /dev/null @@ -1 +0,0 @@ -3404045 diff --git a/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d b/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d deleted file mode 100644 index 272791f402250..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d +++ /dev/null @@ -1 +0,0 @@ -3404045 diff --git a/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e b/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e deleted file mode 100644 index deb8427800ee4..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e +++ /dev/null @@ -1 +0,0 @@ -3404045.5 diff --git a/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 b/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 deleted file mode 100644 index 6f31e8fe55034..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 +++ /dev/null @@ -1 +0,0 @@ -3404045.5044003 diff --git a/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 b/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 deleted file mode 100644 index 6f31e8fe55034..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 +++ /dev/null @@ -1 +0,0 @@ -3404045.5044003 diff --git a/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 b/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 deleted file mode 100644 index 6324d401a069f..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 +++ /dev/null @@ -1 +0,0 @@ -3.14 diff --git a/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 b/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 deleted file mode 100644 index 6324d401a069f..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 +++ /dev/null @@ -1 +0,0 @@ -3.14 diff --git a/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff b/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff deleted file mode 100644 index 603f18cc37bc4..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff +++ /dev/null @@ -1 +0,0 @@ -1355944339.1234567 diff --git a/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 b/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 b/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a b/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a deleted file mode 100644 index 474c8b180aea9..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a +++ /dev/null @@ -1 +0,0 @@ -0.99999999999999999999 diff --git a/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 b/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 b/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 b/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d b/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e b/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 b/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 deleted file mode 100644 index 8d8753f153d7c..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 +++ /dev/null @@ -1,4 +0,0 @@ -3.14 3 -3.14 3 -3.14 3 -3.14 4 diff --git a/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 b/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 deleted file mode 100644 index 8d8753f153d7c..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 +++ /dev/null @@ -1,4 +0,0 @@ -3.14 3 -3.14 3 -3.14 3 -3.14 4 diff --git a/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b b/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b deleted file mode 100644 index 3e290231c27e2..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b +++ /dev/null @@ -1,38 +0,0 @@ -NULL 0 -NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -1 --1.12 -1 --0.333 0 --0.33 0 --0.3 0 -0 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 -0.9999999999999999999999999 1 -1 1 -1 1 -1.12 1 -1.122 1 -2 2 -2 2 -3.14 3 -3.14 3 -3.14 3 -3.14 4 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 b/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 deleted file mode 100644 index 64fa7bca9a81b..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 +++ /dev/null @@ -1,38 +0,0 @@ -1234567890.12345678 1234567890 -200 200 -125.2 125 -124 124 -100 100 -20 20 -10 10 -3.14 4 -3.14 3 -3.14 3 -3.14 3 -2 2 -2 2 -1.122 1 -1.12 1 -1 1 -1 1 -0.9999999999999999999999999 1 -0.333 0 -0.33 0 -0.3 0 -0.2 0 -0.1 0 -0.02 0 -0.01 0 -0 0 -0 0 --0.3 0 --0.33 0 --0.333 0 --1.12 -1 --1.12 -1 --1.122 -11 --1255.49 -1255 --4400 4400 --1234567890.123456789 -1234567890 -NULL 0 -NULL 0 diff --git a/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b b/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b deleted file mode 100644 index 3e290231c27e2..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b +++ /dev/null @@ -1,38 +0,0 @@ -NULL 0 -NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -1 --1.12 -1 --0.333 0 --0.33 0 --0.3 0 -0 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 -0.9999999999999999999999999 1 -1 1 -1 1 -1.12 1 -1.122 1 -2 2 -2 2 -3.14 3 -3.14 3 -3.14 3 -3.14 4 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 b/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 deleted file mode 100644 index 24d34ee5d8c1b..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 +++ /dev/null @@ -1,30 +0,0 @@ -NULL --1234567890.123456789 --4400 --1255.49 --1.122 --1.12 --0.333 --0.33 --0.3 -0 -0.01 -0.02 -0.1 -0.2 -0.3 -0.33 -0.333 -0.9999999999999999999999999 -1 -1.12 -1.122 -2 -3.14 -10 -20 -100 -124 -125.2 -200 -1234567890.12345678 diff --git a/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 b/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 deleted file mode 100644 index e08f588c89461..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 +++ /dev/null @@ -1,30 +0,0 @@ -NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -2 --0.333 0 --0.33 0 --0.3 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 -0.9999999999999999999999999 1 -1 2 -1.12 1 -1.122 1 -2 4 -3.14 13 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 b/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 deleted file mode 100644 index 796707d06b0dd..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 +++ /dev/null @@ -1,17 +0,0 @@ --1234567890 -1234567890.123456789 --1255 -1255.49 --11 -1.122 --1 -2.24 -0 0.33 -1 5.2419999999999999999999999 -2 4 -3 9.42 -4 3.14 -10 10 -20 20 -100 100 -124 124 -125 125.2 -200 200 -4400 -4400 -1234567890 1234567890.12345678 diff --git a/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 b/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 deleted file mode 100644 index 4217ad848170e..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 +++ /dev/null @@ -1,56 +0,0 @@ --1234567890.123456789 -1234567890 -1234567890.123456789 -1234567890 --4400 4400 -4400 4400 --1255.49 -1255 -1255.49 -1255 --1.122 -11 -1.122 -11 --1.12 -1 -1.12 -1 --1.12 -1 -1.12 -1 --1.12 -1 -1.12 -1 --1.12 -1 -1.12 -1 --0.333 0 -0.333 0 --0.33 0 -0.33 0 --0.3 0 -0.3 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0.01 0 0.01 0 -0.02 0 0.02 0 -0.1 0 0.1 0 -0.2 0 0.2 0 -0.3 0 0.3 0 -0.33 0 0.33 0 -0.333 0 0.333 0 -0.9999999999999999999999999 1 0.9999999999999999999999999 1 -1 1 1 1 -1 1 1 1 -1 1 1 1 -1 1 1 1 -1.12 1 1.12 1 -1.122 1 1.122 1 -2 2 2 2 -2 2 2 2 -2 2 2 2 -2 2 2 2 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 4 -3.14 3 3.14 4 -3.14 3 3.14 4 -3.14 4 3.14 3 -3.14 4 3.14 3 -3.14 4 3.14 3 -3.14 4 3.14 4 -10 10 10 10 -20 20 20 20 -100 100 100 100 -124 124 124 124 -125.2 125 125.2 125 -200 200 200 200 -1234567890.12345678 1234567890 1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 b/sql/hive/src/test/resources/golden/decimal_4-2-85c3185beb011f5c1e6856fc773a7484 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 rename to sql/hive/src/test/resources/golden/decimal_4-2-85c3185beb011f5c1e6856fc773a7484 diff --git a/sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 b/sql/hive/src/test/resources/golden/decimal_4-3-1451d7491441c1632fd5f751876cce6e similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 rename to sql/hive/src/test/resources/golden/decimal_4-3-1451d7491441c1632fd5f751876cce6e diff --git a/sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a b/sql/hive/src/test/resources/golden/decimal_4-4-1bf9ff1d72a06c33885ba695adf2511d similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a rename to sql/hive/src/test/resources/golden/decimal_4-4-1bf9ff1d72a06c33885ba695adf2511d diff --git a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e index 3e290231c27e2..f59549a6e4a46 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e +++ b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e @@ -1,5 +1,4 @@ NULL 0 -NULL 0 -1234567890.123456789 -1234567890 -4400 4400 -1255.49 -1255 @@ -11,6 +10,7 @@ NULL 0 -0.3 0 0 0 0 0 +0 0 0.01 0 0.02 0 0.1 0 diff --git a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 index 795a4b567ab7f..6bada475c6d3d 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 +++ b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 @@ -1,5 +1,4 @@ NULL NULL -NULL NULL -1234567890.123456789 -3703703670.370370367 -4400 -13200 -1255.49 -3766.47 @@ -11,6 +10,7 @@ NULL NULL -0.3 -0.9 0 0 0 0 +0 0 0.01 0.03 0.02 0.06 0.1 0.3 diff --git a/sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 b/sql/hive/src/test/resources/golden/decimal_join-0-4668e9dee2cd7a32f2b7311d7cd35508 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 rename to sql/hive/src/test/resources/golden/decimal_join-0-4668e9dee2cd7a32f2b7311d7cd35508 diff --git a/sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c b/sql/hive/src/test/resources/golden/decimal_join-1-5098974222b22a21ed847c7906df9313 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c rename to sql/hive/src/test/resources/golden/decimal_join-1-5098974222b22a21ed847c7906df9313 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b b/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b deleted file mode 100644 index 3d9e792183f3c..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b +++ /dev/null @@ -1,75 +0,0 @@ -NULL -NULL -NULL -NULL -NULL -NULL -NULL -NULL --99999999999999999999999999999999999999 --999999999999999999999999999999999999 --99999999999999999999999999999999999 --0.0000000000000000000000000000000000001 -0 -0.0000000000000000000000000000000000001 -0.123456789012345 -0.1234567890123456789012345678901234578 -1.234567890123456 -1.2345678901234567890123456789012345678 -12.34567890123456 -12.345678901234567890123456789012345678 -123.4567890123456 -123.45678901234567890123456789012345678 -1234.567890123456 -1234.5678901234567890123456789012345678 -12345.67890123456 -12345.678901234567890123456789012345678 -123456.7890123456 -123456.78901234567890123456789012345678 -1234567.890123456 -1234567.8901234567890123456789012345678 -12345678.90123456 -12345678.901234567890123456789012345678 -123456789.0123456 -123456789.01234567890123456789012345678 -1234567890.123456 -1234567890.1234567890123456789012345678 -12345678901.23456 -12345678901.234567890123456789012345678 -123456789012.3456 -123456789012.34567890123456789012345678 -1234567890123.456 -1234567890123.4567890123456789012345678 -12345678901234.56 -12345678901234.567890123456789012345678 -123456789012345.6 -123456789012345.67890123456789012345678 -1234567890123456.7890123456789012345678 -12345678901234567.890123456789012345678 -123456789012345678.90123456789012345678 -1234567890123456789.0123456789012345678 -12345678901234567890.123456789012345678 -123456789012345678901.23456789012345678 -1234567890123456789012.3456789012345678 -12345678901234567890123.456789012345678 -123456789012345678901234.56789012345678 -1234567890123456789012345.6789012345678 -12345678901234567890123456.789012345678 -123456789012345678901234567.89012345678 -1234567890123456789012345678.9012345678 -12345678901234567890123456789.012345678 -123456789012345678901234567890.12345678 -1234567890123456789012345678901.2345678 -12345678901234567890123456789012.345678 -123456789012345678901234567890123.45678 -1234567890123456789012345678901234.5678 -12345678901234567890123456789012345.678 -99999999999999999999999999999999999 -123456789012345678901234567890123456.78 -999999999999999999999999999999999999 -12345678901234567890123456789012345678 -12345678901234567890123456789012345678 -12345678901234567890123456789012345678 -12345678901234567890123456789012345678 -12345678901234567890123456789012345678 -99999999999999999999999999999999999999 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc b/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc deleted file mode 100644 index 9853ce72ed8c3..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL --99999999999999999999999999999999999999 -99999999999999999999999999999999999998 NULL --999999999999999999999999999999999999 -999999999999999999999999999999999998 -1000000000000000000000000000000000000 --99999999999999999999999999999999999 -99999999999999999999999999999999998 -100000000000000000000000000000000000 --0.0000000000000000000000000000000000001 0.9999999999999999999999999999999999999 -1.0000000000000000000000000000000000001 -0 1 -1 -0.0000000000000000000000000000000000001 1.0000000000000000000000000000000000001 -0.9999999999999999999999999999999999999 -0.123456789012345 1.123456789012345 -0.876543210987655 -0.1234567890123456789012345678901234578 1.1234567890123456789012345678901234578 -0.8765432109876543210987654321098765422 -1.234567890123456 2.234567890123456 0.234567890123456 -1.2345678901234567890123456789012345678 2.2345678901234567890123456789012345678 0.2345678901234567890123456789012345678 -12.34567890123456 13.34567890123456 11.34567890123456 -12.345678901234567890123456789012345678 13.345678901234567890123456789012345678 11.345678901234567890123456789012345678 -123.4567890123456 124.4567890123456 122.4567890123456 -123.45678901234567890123456789012345678 124.45678901234567890123456789012345678 122.45678901234567890123456789012345678 -1234.567890123456 1235.567890123456 1233.567890123456 -1234.5678901234567890123456789012345678 1235.5678901234567890123456789012345678 1233.5678901234567890123456789012345678 -12345.67890123456 12346.67890123456 12344.67890123456 -12345.678901234567890123456789012345678 12346.678901234567890123456789012345678 12344.678901234567890123456789012345678 -123456.7890123456 123457.7890123456 123455.7890123456 -123456.78901234567890123456789012345678 123457.78901234567890123456789012345678 123455.78901234567890123456789012345678 -1234567.890123456 1234568.890123456 1234566.890123456 -1234567.8901234567890123456789012345678 1234568.8901234567890123456789012345678 1234566.8901234567890123456789012345678 -12345678.90123456 12345679.90123456 12345677.90123456 -12345678.901234567890123456789012345678 12345679.901234567890123456789012345678 12345677.901234567890123456789012345678 -123456789.0123456 123456790.0123456 123456788.0123456 -123456789.01234567890123456789012345678 123456790.01234567890123456789012345678 123456788.01234567890123456789012345678 -1234567890.123456 1234567891.123456 1234567889.123456 -1234567890.1234567890123456789012345678 1234567891.1234567890123456789012345678 1234567889.1234567890123456789012345678 -12345678901.23456 12345678902.23456 12345678900.23456 -12345678901.234567890123456789012345678 12345678902.234567890123456789012345678 12345678900.234567890123456789012345678 -123456789012.3456 123456789013.3456 123456789011.3456 -123456789012.34567890123456789012345678 123456789013.34567890123456789012345678 123456789011.34567890123456789012345678 -1234567890123.456 1234567890124.456 1234567890122.456 -1234567890123.4567890123456789012345678 1234567890124.4567890123456789012345678 1234567890122.4567890123456789012345678 -12345678901234.56 12345678901235.56 12345678901233.56 -12345678901234.567890123456789012345678 12345678901235.567890123456789012345678 12345678901233.567890123456789012345678 -123456789012345.6 123456789012346.6 123456789012344.6 -123456789012345.67890123456789012345678 123456789012346.67890123456789012345678 123456789012344.67890123456789012345678 -1234567890123456.7890123456789012345678 1234567890123457.7890123456789012345678 1234567890123455.7890123456789012345678 -12345678901234567.890123456789012345678 12345678901234568.890123456789012345678 12345678901234566.890123456789012345678 -123456789012345678.90123456789012345678 123456789012345679.90123456789012345678 123456789012345677.90123456789012345678 -1234567890123456789.0123456789012345678 1234567890123456790.0123456789012345678 1234567890123456788.0123456789012345678 -12345678901234567890.123456789012345678 12345678901234567891.123456789012345678 12345678901234567889.123456789012345678 -123456789012345678901.23456789012345678 123456789012345678902.23456789012345678 123456789012345678900.23456789012345678 -1234567890123456789012.3456789012345678 1234567890123456789013.3456789012345678 1234567890123456789011.3456789012345678 -12345678901234567890123.456789012345678 12345678901234567890124.456789012345678 12345678901234567890122.456789012345678 -123456789012345678901234.56789012345678 123456789012345678901235.56789012345678 123456789012345678901233.56789012345678 -1234567890123456789012345.6789012345678 1234567890123456789012346.6789012345678 1234567890123456789012344.6789012345678 -12345678901234567890123456.789012345678 12345678901234567890123457.789012345678 12345678901234567890123455.789012345678 -123456789012345678901234567.89012345678 123456789012345678901234568.89012345678 123456789012345678901234566.89012345678 -1234567890123456789012345678.9012345678 1234567890123456789012345679.9012345678 1234567890123456789012345677.9012345678 -12345678901234567890123456789.012345678 12345678901234567890123456790.012345678 12345678901234567890123456788.012345678 -123456789012345678901234567890.12345678 123456789012345678901234567891.12345678 123456789012345678901234567889.12345678 -1234567890123456789012345678901.2345678 1234567890123456789012345678902.2345678 1234567890123456789012345678900.2345678 -12345678901234567890123456789012.345678 12345678901234567890123456789013.345678 12345678901234567890123456789011.345678 -123456789012345678901234567890123.45678 123456789012345678901234567890124.45678 123456789012345678901234567890122.45678 -1234567890123456789012345678901234.5678 1234567890123456789012345678901235.5678 1234567890123456789012345678901233.5678 -12345678901234567890123456789012345.678 12345678901234567890123456789012346.678 12345678901234567890123456789012344.678 -99999999999999999999999999999999999 100000000000000000000000000000000000 99999999999999999999999999999999998 -123456789012345678901234567890123456.78 123456789012345678901234567890123457.78 123456789012345678901234567890123455.78 -999999999999999999999999999999999999 1000000000000000000000000000000000000 999999999999999999999999999999999998 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -99999999999999999999999999999999999999 NULL 99999999999999999999999999999999999998 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef b/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef deleted file mode 100644 index 7cc75c789dee1..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL --99999999999999999999999999999999999999 NULL -33333333333333333333333333333333333333 --999999999999999999999999999999999999 -1999999999999999999999999999999999998 -333333333333333333333333333333333333 --99999999999999999999999999999999999 -199999999999999999999999999999999998 -33333333333333333333333333333333333 --0.0000000000000000000000000000000000001 -0.0000000000000000000000000000000000002 0 -0 0 0 -0.0000000000000000000000000000000000001 0.0000000000000000000000000000000000002 0 -0.123456789012345 0.24691357802469 0.041152263004115 -0.1234567890123456789012345678901234578 0.2469135780246913578024691357802469156 0.0411522630041152263004115226300411526 -1.234567890123456 2.469135780246912 0.411522630041152 -1.2345678901234567890123456789012345678 2.4691357802469135780246913578024691356 0.4115226300411522630041152263004115226 -12.34567890123456 24.69135780246912 4.11522630041152 -12.345678901234567890123456789012345678 24.691357802469135780246913578024691356 4.115226300411522630041152263004115226 -123.4567890123456 246.9135780246912 41.1522630041152 -123.45678901234567890123456789012345678 246.91357802469135780246913578024691356 41.15226300411522630041152263004115226 -1234.567890123456 2469.135780246912 411.522630041152 -1234.5678901234567890123456789012345678 2469.1357802469135780246913578024691356 411.5226300411522630041152263004115226 -12345.67890123456 24691.35780246912 4115.22630041152 -12345.678901234567890123456789012345678 24691.357802469135780246913578024691356 4115.226300411522630041152263004115226 -123456.7890123456 246913.5780246912 41152.2630041152 -123456.78901234567890123456789012345678 246913.57802469135780246913578024691356 41152.26300411522630041152263004115226 -1234567.890123456 2469135.780246912 411522.630041152 -1234567.8901234567890123456789012345678 2469135.7802469135780246913578024691356 411522.6300411522630041152263004115226 -12345678.90123456 24691357.80246912 4115226.30041152 -12345678.901234567890123456789012345678 24691357.802469135780246913578024691356 4115226.300411522630041152263004115226 -123456789.0123456 246913578.0246912 41152263.0041152 -123456789.01234567890123456789012345678 246913578.02469135780246913578024691356 41152263.00411522630041152263004115226 -1234567890.123456 2469135780.246912 411522630.041152 -1234567890.1234567890123456789012345678 2469135780.2469135780246913578024691356 411522630.0411522630041152263004115226 -12345678901.23456 24691357802.46912 4115226300.41152 -12345678901.234567890123456789012345678 24691357802.469135780246913578024691356 4115226300.411522630041152263004115226 -123456789012.3456 246913578024.6912 41152263004.1152 -123456789012.34567890123456789012345678 246913578024.69135780246913578024691356 41152263004.11522630041152263004115226 -1234567890123.456 2469135780246.912 411522630041.152 -1234567890123.4567890123456789012345678 2469135780246.9135780246913578024691356 411522630041.1522630041152263004115226 -12345678901234.56 24691357802469.12 4115226300411.52 -12345678901234.567890123456789012345678 24691357802469.135780246913578024691356 4115226300411.522630041152263004115226 -123456789012345.6 246913578024691.2 41152263004115.2 -123456789012345.67890123456789012345678 246913578024691.35780246913578024691356 41152263004115.22630041152263004115226 -1234567890123456.7890123456789012345678 2469135780246913.5780246913578024691356 411522630041152.2630041152263004115226 -12345678901234567.890123456789012345678 24691357802469135.780246913578024691356 4115226300411522.630041152263004115226 -123456789012345678.90123456789012345678 246913578024691357.80246913578024691356 41152263004115226.30041152263004115226 -1234567890123456789.0123456789012345678 2469135780246913578.0246913578024691356 411522630041152263.0041152263004115226 -12345678901234567890.123456789012345678 24691357802469135780.246913578024691356 4115226300411522630.041152263004115226 -123456789012345678901.23456789012345678 246913578024691357802.46913578024691356 41152263004115226300.41152263004115226 -1234567890123456789012.3456789012345678 2469135780246913578024.6913578024691356 411522630041152263004.1152263004115226 -12345678901234567890123.456789012345678 24691357802469135780246.913578024691356 4115226300411522630041.152263004115226 -123456789012345678901234.56789012345678 246913578024691357802469.13578024691356 41152263004115226300411.52263004115226 -1234567890123456789012345.6789012345678 2469135780246913578024691.3578024691356 411522630041152263004115.2263004115226 -12345678901234567890123456.789012345678 24691357802469135780246913.578024691356 4115226300411522630041152.263004115226 -123456789012345678901234567.89012345678 246913578024691357802469135.78024691356 41152263004115226300411522.63004115226 -1234567890123456789012345678.9012345678 2469135780246913578024691357.8024691356 411522630041152263004115226.3004115226 -12345678901234567890123456789.012345678 24691357802469135780246913578.024691356 4115226300411522630041152263.004115226 -123456789012345678901234567890.12345678 246913578024691357802469135780.24691356 41152263004115226300411522630.04115226 -1234567890123456789012345678901.2345678 2469135780246913578024691357802.4691356 411522630041152263004115226300.4115226 -12345678901234567890123456789012.345678 24691357802469135780246913578024.691356 4115226300411522630041152263004.115226 -123456789012345678901234567890123.45678 246913578024691357802469135780246.91356 41152263004115226300411522630041.15226 -1234567890123456789012345678901234.5678 2469135780246913578024691357802469.1356 411522630041152263004115226300411.5226 -12345678901234567890123456789012345.678 24691357802469135780246913578024691.356 4115226300411522630041152263004115.226 -99999999999999999999999999999999999 199999999999999999999999999999999998 33333333333333333333333333333333333 -123456789012345678901234567890123456.78 246913578024691357802469135780246913.56 41152263004115226300411522630041152.26 -999999999999999999999999999999999999 1999999999999999999999999999999999998 333333333333333333333333333333333333 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -99999999999999999999999999999999999999 NULL 33333333333333333333333333333333333333 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 b/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 deleted file mode 100644 index c40875630d1b2..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL --99999999999999999999999999999999999999 -11111111111111111111111111111111111111 --999999999999999999999999999999999999 -111111111111111111111111111111111111 --99999999999999999999999999999999999 -11111111111111111111111111111111111 --0.0000000000000000000000000000000000001 0 -0 0 -0.0000000000000000000000000000000000001 0 -0.123456789012345 0.0137174210013716666666666666666666667 -0.1234567890123456789012345678901234578 0.0137174210013717421001371742100137175 -1.234567890123456 0.1371742100137173333333333333333333333 -1.2345678901234567890123456789012345678 0.1371742100137174210013717421001371742 -12.34567890123456 1.3717421001371733333333333333333333333 -12.345678901234567890123456789012345678 1.371742100137174210013717421001371742 -123.4567890123456 13.717421001371733333333333333333333333 -123.45678901234567890123456789012345678 13.71742100137174210013717421001371742 -1234.567890123456 137.17421001371733333333333333333333333 -1234.5678901234567890123456789012345678 137.1742100137174210013717421001371742 -12345.67890123456 1371.7421001371733333333333333333333333 -12345.678901234567890123456789012345678 1371.742100137174210013717421001371742 -123456.7890123456 13717.421001371733333333333333333333333 -123456.78901234567890123456789012345678 13717.42100137174210013717421001371742 -1234567.890123456 137174.21001371733333333333333333333333 -1234567.8901234567890123456789012345678 137174.2100137174210013717421001371742 -12345678.90123456 1371742.1001371733333333333333333333333 -12345678.901234567890123456789012345678 1371742.100137174210013717421001371742 -123456789.0123456 13717421.001371733333333333333333333333 -123456789.01234567890123456789012345678 13717421.00137174210013717421001371742 -1234567890.123456 137174210.01371733333333333333333333333 -1234567890.1234567890123456789012345678 137174210.0137174210013717421001371742 -12345678901.23456 1371742100.1371733333333333333333333333 -12345678901.234567890123456789012345678 1371742100.137174210013717421001371742 -123456789012.3456 13717421001.371733333333333333333333333 -123456789012.34567890123456789012345678 13717421001.37174210013717421001371742 -1234567890123.456 137174210013.71733333333333333333333333 -1234567890123.4567890123456789012345678 137174210013.7174210013717421001371742 -12345678901234.56 1371742100137.1733333333333333333333333 -12345678901234.567890123456789012345678 1371742100137.174210013717421001371742 -123456789012345.6 13717421001371.733333333333333333333333 -123456789012345.67890123456789012345678 13717421001371.74210013717421001371742 -1234567890123456.7890123456789012345678 137174210013717.4210013717421001371742 -12345678901234567.890123456789012345678 1371742100137174.210013717421001371742 -123456789012345678.90123456789012345678 13717421001371742.10013717421001371742 -1234567890123456789.0123456789012345678 137174210013717421.0013717421001371742 -12345678901234567890.123456789012345678 1371742100137174210.013717421001371742 -123456789012345678901.23456789012345678 13717421001371742100.13717421001371742 -1234567890123456789012.3456789012345678 137174210013717421001.3717421001371742 -12345678901234567890123.456789012345678 1371742100137174210013.717421001371742 -123456789012345678901234.56789012345678 13717421001371742100137.17421001371742 -1234567890123456789012345.6789012345678 137174210013717421001371.7421001371742 -12345678901234567890123456.789012345678 1371742100137174210013717.421001371742 -123456789012345678901234567.89012345678 13717421001371742100137174.21001371742 -1234567890123456789012345678.9012345678 137174210013717421001371742.1001371742 -12345678901234567890123456789.012345678 1371742100137174210013717421.001371742 -123456789012345678901234567890.12345678 13717421001371742100137174210.01371742 -1234567890123456789012345678901.2345678 137174210013717421001371742100.1371742 -12345678901234567890123456789012.345678 1371742100137174210013717421001.371742 -123456789012345678901234567890123.45678 13717421001371742100137174210013.71742 -1234567890123456789012345678901234.5678 137174210013717421001371742100137.1742 -12345678901234567890123456789012345.678 1371742100137174210013717421001371.742 -99999999999999999999999999999999999 11111111111111111111111111111111111 -123456789012345678901234567890123456.78 13717421001371742100137174210013717.42 -999999999999999999999999999999999999 111111111111111111111111111111111111 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -99999999999999999999999999999999999999 11111111111111111111111111111111111111 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 b/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 deleted file mode 100644 index bd23d17293f79..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL --99999999999999999999999999999999999999 -3703703703703703703703703703703703703.7 --999999999999999999999999999999999999 -37037037037037037037037037037037037 --99999999999999999999999999999999999 -3703703703703703703703703703703703.6667 --0.0000000000000000000000000000000000001 0 -0 0 -0.0000000000000000000000000000000000001 0 -0.123456789012345 0.0045724736671238888888888888888888889 -0.1234567890123456789012345678901234578 0.0045724736671239140333790580700045725 -1.234567890123456 0.0457247366712391111111111111111111111 -1.2345678901234567890123456789012345678 0.0457247366712391403337905807000457247 -12.34567890123456 0.4572473667123911111111111111111111111 -12.345678901234567890123456789012345678 0.4572473667123914033379058070004572473 -123.4567890123456 4.5724736671239111111111111111111111111 -123.45678901234567890123456789012345678 4.5724736671239140333790580700045724733 -1234.567890123456 45.724736671239111111111111111111111111 -1234.5678901234567890123456789012345678 45.724736671239140333790580700045724733 -12345.67890123456 457.24736671239111111111111111111111111 -12345.678901234567890123456789012345678 457.24736671239140333790580700045724733 -123456.7890123456 4572.4736671239111111111111111111111111 -123456.78901234567890123456789012345678 4572.4736671239140333790580700045724733 -1234567.890123456 45724.736671239111111111111111111111111 -1234567.8901234567890123456789012345678 45724.736671239140333790580700045724733 -12345678.90123456 457247.36671239111111111111111111111111 -12345678.901234567890123456789012345678 457247.36671239140333790580700045724733 -123456789.0123456 4572473.6671239111111111111111111111111 -123456789.01234567890123456789012345678 4572473.6671239140333790580700045724733 -1234567890.123456 45724736.671239111111111111111111111111 -1234567890.1234567890123456789012345678 45724736.671239140333790580700045724733 -12345678901.23456 457247366.71239111111111111111111111111 -12345678901.234567890123456789012345678 457247366.71239140333790580700045724733 -123456789012.3456 4572473667.1239111111111111111111111111 -123456789012.34567890123456789012345678 4572473667.1239140333790580700045724733 -1234567890123.456 45724736671.239111111111111111111111111 -1234567890123.4567890123456789012345678 45724736671.239140333790580700045724733 -12345678901234.56 457247366712.39111111111111111111111111 -12345678901234.567890123456789012345678 457247366712.39140333790580700045724733 -123456789012345.6 4572473667123.9111111111111111111111111 -123456789012345.67890123456789012345678 4572473667123.9140333790580700045724733 -1234567890123456.7890123456789012345678 45724736671239.140333790580700045724733 -12345678901234567.890123456789012345678 457247366712391.40333790580700045724733 -123456789012345678.90123456789012345678 4572473667123914.0333790580700045724733 -1234567890123456789.0123456789012345678 45724736671239140.333790580700045724733 -12345678901234567890.123456789012345678 457247366712391403.33790580700045724733 -123456789012345678901.23456789012345678 4572473667123914033.3790580700045724733 -1234567890123456789012.3456789012345678 45724736671239140333.790580700045724733 -12345678901234567890123.456789012345678 457247366712391403337.90580700045724733 -123456789012345678901234.56789012345678 4572473667123914033379.0580700045724733 -1234567890123456789012345.6789012345678 45724736671239140333790.580700045724733 -12345678901234567890123456.789012345678 457247366712391403337905.80700045724733 -123456789012345678901234567.89012345678 4572473667123914033379058.0700045724733 -1234567890123456789012345678.9012345678 45724736671239140333790580.700045724733 -12345678901234567890123456789.012345678 457247366712391403337905807.00045724733 -123456789012345678901234567890.12345678 4572473667123914033379058070.0045724733 -1234567890123456789012345678901.2345678 45724736671239140333790580700.045724733 -12345678901234567890123456789012.345678 457247366712391403337905807000.45724733 -123456789012345678901234567890123.45678 4572473667123914033379058070004.5724733 -1234567890123456789012345678901234.5678 45724736671239140333790580700045.724733 -12345678901234567890123456789012345.678 457247366712391403337905807000457.24733 -99999999999999999999999999999999999 3703703703703703703703703703703703.6667 -123456789012345678901234567890123456.78 4572473667123914033379058070004572.4733 -999999999999999999999999999999999999 37037037037037037037037037037037037 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -99999999999999999999999999999999999999 3703703703703703703703703703703703703.7 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e b/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e deleted file mode 100644 index c1e0db0174c63..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL --99999999999999999999999999999999999999 NULL --999999999999999999999999999999999999 NULL --99999999999999999999999999999999999 NULL --0.0000000000000000000000000000000000001 NULL -0 0 -0.0000000000000000000000000000000000001 NULL -0.123456789012345 0.015241578753238669120562399025 -0.1234567890123456789012345678901234578 NULL -1.234567890123456 1.524157875323881726870921383936 -1.2345678901234567890123456789012345678 NULL -12.34567890123456 152.4157875323881726870921383936 -12.345678901234567890123456789012345678 NULL -123.4567890123456 15241.57875323881726870921383936 -123.45678901234567890123456789012345678 NULL -1234.567890123456 1524157.875323881726870921383936 -1234.5678901234567890123456789012345678 NULL -12345.67890123456 152415787.5323881726870921383936 -12345.678901234567890123456789012345678 NULL -123456.7890123456 15241578753.23881726870921383936 -123456.78901234567890123456789012345678 NULL -1234567.890123456 1524157875323.881726870921383936 -1234567.8901234567890123456789012345678 NULL -12345678.90123456 152415787532388.1726870921383936 -12345678.901234567890123456789012345678 NULL -123456789.0123456 15241578753238817.26870921383936 -123456789.01234567890123456789012345678 NULL -1234567890.123456 1524157875323881726.870921383936 -1234567890.1234567890123456789012345678 NULL -12345678901.23456 152415787532388172687.0921383936 -12345678901.234567890123456789012345678 NULL -123456789012.3456 15241578753238817268709.21383936 -123456789012.34567890123456789012345678 NULL -1234567890123.456 1524157875323881726870921.383936 -1234567890123.4567890123456789012345678 NULL -12345678901234.56 152415787532388172687092138.3936 -12345678901234.567890123456789012345678 NULL -123456789012345.6 15241578753238817268709213839.36 -123456789012345.67890123456789012345678 NULL -1234567890123456.7890123456789012345678 NULL -12345678901234567.890123456789012345678 NULL -123456789012345678.90123456789012345678 NULL -1234567890123456789.0123456789012345678 NULL -12345678901234567890.123456789012345678 NULL -123456789012345678901.23456789012345678 NULL -1234567890123456789012.3456789012345678 NULL -12345678901234567890123.456789012345678 NULL -123456789012345678901234.56789012345678 NULL -1234567890123456789012345.6789012345678 NULL -12345678901234567890123456.789012345678 NULL -123456789012345678901234567.89012345678 NULL -1234567890123456789012345678.9012345678 NULL -12345678901234567890123456789.012345678 NULL -123456789012345678901234567890.12345678 NULL -1234567890123456789012345678901.2345678 NULL -12345678901234567890123456789012.345678 NULL -123456789012345678901234567890123.45678 NULL -1234567890123456789012345678901234.5678 NULL -12345678901234567890123456789012345.678 NULL -99999999999999999999999999999999999 NULL -123456789012345678901234567890123456.78 NULL -999999999999999999999999999999999999 NULL -12345678901234567890123456789012345678 NULL -12345678901234567890123456789012345678 NULL -12345678901234567890123456789012345678 NULL -12345678901234567890123456789012345678 NULL -12345678901234567890123456789012345678 NULL -99999999999999999999999999999999999999 NULL diff --git a/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab b/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab deleted file mode 100644 index 81af0e4cd3ab8..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab +++ /dev/null @@ -1 +0,0 @@ -NULL NULL diff --git a/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c b/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c +++ b/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be b/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be index ded23df148827..3aa0e30600f3c 100644 --- a/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be +++ b/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be @@ -1 +1 @@ -ds=__HIVE_DEFAULT_PARTITION__ \ No newline at end of file +ds=__HIVE_DEFAULT_PARTITION__ diff --git a/sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 b/sql/hive/src/test/resources/golden/delimiter-1-121ff21e6931a82235de8301118cbed8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 rename to sql/hive/src/test/resources/golden/delimiter-1-121ff21e6931a82235de8301118cbed8 diff --git a/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 b/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 index f438072c76b5f..424a2fee06987 100644 --- a/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 +++ b/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 @@ -1,3 +1,3 @@ 35 40 48 32 -100100 40 \ No newline at end of file +100100 40 diff --git a/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 b/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 index f438072c76b5f..424a2fee06987 100644 --- a/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 +++ b/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 @@ -1,3 +1,3 @@ 35 40 48 32 -100100 40 \ No newline at end of file +100100 40 diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 b/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 deleted file mode 100644 index d980efc81b947..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 +++ /dev/null @@ -1,6 +0,0 @@ -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 b/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 deleted file mode 100644 index 01b9151074b22..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 14 11:42:35 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/test_table -Table Type: MANAGED_TABLE -Table Parameters: - comment table comment\ntwo lines - transient_lastDdlTime 1389728555 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 b/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a b/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a deleted file mode 100644 index c56a79e4f322e..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a +++ /dev/null @@ -1 +0,0 @@ -{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1","comment":"Test database","database":"jsondb1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 b/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 deleted file mode 100644 index aa08c38c68d1d..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 +++ /dev/null @@ -1 +0,0 @@ -{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1","params":{"id":"jsondb1"},"comment":"Test database","database":"jsondb1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 deleted file mode 100644 index 513aeaab1dc66..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 +++ /dev/null @@ -1 +0,0 @@ -{"databases":["default","jsondb1"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c b/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c deleted file mode 100644 index 606069d6291b4..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c +++ /dev/null @@ -1 +0,0 @@ -{"databases":["jsondb1"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a b/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a deleted file mode 100644 index bea7c01440c46..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a +++ /dev/null @@ -1 +0,0 @@ -{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1.db","database":"jsondb1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 b/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 deleted file mode 100644 index bea7c01440c46..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 +++ /dev/null @@ -1 +0,0 @@ -{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1.db","database":"jsondb1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 index 381821184d693..1ed0de6860c08 100644 --- a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 +++ b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 @@ -1,19 +1,19 @@ # col_name data_type comment -key int None +key int # Partition Information # col_name data_type comment -value string None +value string # Detailed Partition Information Partition Value: [val_86] Database: default Table: view_partitioned -CreateTime: Fri Feb 07 15:09:16 PST 2014 +CreateTime: Tue Oct 21 01:26:15 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Location: null Partition Parameters: - transient_lastDdlTime 1391814556 + transient_lastDdlTime 1413879975 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 deleted file mode 100644 index 7b51873776ad8..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 +++ /dev/null @@ -1 +0,0 @@ -{"columns":[{"name":"key","type":"int"}]} diff --git a/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf b/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf b/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 b/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 deleted file mode 100644 index 35fac1b6f2579..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 +++ /dev/null @@ -1,12 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2_abcdefghiklmnopqrstuvxyz string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that is - greater than 80 chars and is - likely to spill into multiple - lines \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 b/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 b/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 deleted file mode 100644 index b57f8955ca397..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 +++ /dev/null @@ -1,24 +0,0 @@ -col_name data_type comment - -col1 int col1 one - line - comment -col2_abcdefghiklmnopqrstuvxyz string col2 - two lines - comment -col3 string col3 - three - lines - comment -col4 string col4 very - long - comment - that is - greater - than 80 - chars and - is likely - to spill - into - multiple - lines \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 b/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 b/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 deleted file mode 100644 index 3b7fe3c133089..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 +++ /dev/null @@ -1,37 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that - is greater than 80 chars - and is likely to spill into - multiple lines -col5 string col5 very long multi-line - comment where each line is - very long by itself and is - likely to spill - into multiple lines. Lorem - ipsum dolor sit amet, - consectetur adipiscing - elit. Proin in dolor nisl, - sodales - adipiscing tortor. Integer - venenatis -col6 string This comment has a very - long single word ABCDEFGHIJ - KLMNOPQRSTUVXYZabcdefghijkl - mnopqrstuvzxyz123 which - will not fit in a line by - itself for small column - widths. -col7_nocomment string None -ds string None - -# Partition Information -col_name data_type comment - -ds string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 b/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 deleted file mode 100644 index 49175da27357f..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 +++ /dev/null @@ -1,23 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that is greater than 80 - chars and is likely to spill into multiple - lines -col5 string col5 very long multi-line comment where each - line is very long by itself and is likely to - spill - into multiple lines. Lorem ipsum dolor sit - amet, consectetur adipiscing elit. Proin in - dolor nisl, sodales - adipiscing tortor. Integer venenatis -col6 string This comment has a very long single word ABCDEF - GHIJKLMNOPQRSTUVXYZabcdefghijklmnopqrstuvzxyz12 - 3 which will not fit in a line by itself for - small column widths. -col7_nocomment string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 b/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 deleted file mode 100644 index c3e77e079a9d4..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 +++ /dev/null @@ -1,14 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that is greater than 80 chars and is likely to spill into multiple lines -col5 string col5 very long multi-line comment where each line is very long by itself and is likely to spill - into multiple lines. Lorem ipsum dolor sit amet, consectetur adipiscing elit. Proin in dolor nisl, sodales - adipiscing tortor. Integer venenatis -col6 string This comment has a very long single word ABCDEFGHIJKLMNOPQRSTUVXYZabcdefghijklmnopqrstuvzxyz123 which will not fit in a line by itself for small column widths. -col7_nocomment string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 b/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 deleted file mode 100644 index 452f75a11fdd0..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 +++ /dev/null @@ -1,44 +0,0 @@ -col_name data_type comment - -col1 int col1 one line - comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long - comment that is - greater than 80 - chars and is - likely to spill - into multiple - lines -col5 string col5 very long - multi-line - comment where - each line is very - long by itself - and is likely to - spill - into multiple - lines. Lorem - ipsum dolor sit - amet, consectetur - adipiscing elit. - Proin in dolor - nisl, sodales - adipiscing - tortor. Integer - venenatis -col6 string This comment has - a very long - single word ABCDE - FGHIJKLMNOPQRSTUV - XYZabcdefghijklmn - opqrstuvzxyz123 - which will not - fit in a line by - itself for small - column widths. -col7_nocomment string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 b/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 deleted file mode 100644 index ee5a10c85057a..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 +++ /dev/null @@ -1,31 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that - is greater than 80 chars - and is likely to spill into - multiple lines -col5 string col5 very long multi-line - comment where each line is - very long by itself and is - likely to spill - into multiple lines. Lorem - ipsum dolor sit amet, - consectetur adipiscing - elit. Proin in dolor nisl, - sodales - adipiscing tortor. Integer - venenatis -col6 string This comment has a very - long single word ABCDEFGHIJ - KLMNOPQRSTUVXYZabcdefghijkl - mnopqrstuvzxyz123 which - will not fit in a line by - itself for small column - widths. -col7_nocomment string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 b/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 deleted file mode 100644 index 4184ce21dc079..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 +++ /dev/null @@ -1,12 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -Detailed Table Information Table(tableName:t1, dbName:db1, owner:marmbrus, createTime:1389728588, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728588}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f b/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f deleted file mode 100644 index c94d6dcb90042..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f +++ /dev/null @@ -1,33 +0,0 @@ -# col_name data_type comment - -key1 int None -value1 string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -# Detailed Table Information -Database: db1 -Owner: marmbrus -CreateTime: Tue Jan 14 11:43:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1389728588 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 b/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 b/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 b/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 deleted file mode 100644 index f3d242157dd98..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 +++ /dev/null @@ -1,3 +0,0 @@ -# col_name data_type comment - -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 b/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 b/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 b/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 deleted file mode 100644 index f3d242157dd98..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 +++ /dev/null @@ -1,3 +0,0 @@ -# col_name data_type comment - -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 b/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa b/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a b/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a deleted file mode 100644 index f3d242157dd98..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a +++ /dev/null @@ -1,3 +0,0 @@ -# col_name data_type comment - -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 b/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 deleted file mode 100644 index 4f76eaca6cd8b..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 +++ /dev/null @@ -1,10 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f b/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f deleted file mode 100644 index aa25ca5a29dd3..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f +++ /dev/null @@ -1,12 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -Detailed Partition Information Partition(values:[4, 5], dbName:db1, tableName:t1, createTime:1389728588, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728588}) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 b/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 deleted file mode 100644 index 311870f6ad6b0..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key1 int None -value1 string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -# Detailed Partition Information -Partition Value: [4, 5] -Database: db1 -Table: t1 -CreateTime: Tue Jan 14 11:43:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5 -Partition Parameters: - transient_lastDdlTime 1389728588 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 b/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 deleted file mode 100644 index 4f76eaca6cd8b..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 +++ /dev/null @@ -1,10 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e b/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e deleted file mode 100644 index aa25ca5a29dd3..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e +++ /dev/null @@ -1,12 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -Detailed Partition Information Partition(values:[4, 5], dbName:db1, tableName:t1, createTime:1389728588, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728588}) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec b/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec deleted file mode 100644 index 311870f6ad6b0..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key1 int None -value1 string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -# Detailed Partition Information -Partition Value: [4, 5] -Database: db1 -Table: t1 -CreateTime: Tue Jan 14 11:43:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5 -Partition Parameters: - transient_lastDdlTime 1389728588 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 b/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 deleted file mode 100644 index 4f76eaca6cd8b..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 +++ /dev/null @@ -1,10 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d b/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d deleted file mode 100644 index 4184ce21dc079..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d +++ /dev/null @@ -1,12 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -Detailed Table Information Table(tableName:t1, dbName:db1, owner:marmbrus, createTime:1389728588, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728588}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 b/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 deleted file mode 100644 index c94d6dcb90042..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 +++ /dev/null @@ -1,33 +0,0 @@ -# col_name data_type comment - -key1 int None -value1 string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -# Detailed Table Information -Database: db1 -Owner: marmbrus -CreateTime: Tue Jan 14 11:43:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1389728588 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b b/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b deleted file mode 100644 index 4f76eaca6cd8b..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b +++ /dev/null @@ -1,10 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c deleted file mode 100644 index f8bc404bf7308..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c +++ /dev/null @@ -1 +0,0 @@ -{"tables":["jsontable","src","srcpart"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a b/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a deleted file mode 100644 index 5895645dbbb50..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a +++ /dev/null @@ -1 +0,0 @@ -{"tables":["jsontable"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 b/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 deleted file mode 100644 index 353bf2df92f18..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 +++ /dev/null @@ -1 +0,0 @@ -{"tables":[]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 b/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 deleted file mode 100644 index 96c1178ae6eab..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 +++ /dev/null @@ -1 +0,0 @@ -{"columns":[{"name":"key","type":"int"},{"name":"value","type":"string"}]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 b/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 deleted file mode 100644 index 4cf10d1d762b0..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 +++ /dev/null @@ -1 +0,0 @@ -{"columns":[{"name":"key","type":"int"},{"name":"value","type":"string"}],"tableInfo":{"owner":"marmbrus","parameters":{"id":"jsontable","last_modified_by":"marmbrus","last_modified_time":"1389728616","transient_lastDdlTime":"1389728616","comment":"json table"},"createTime":1389728615,"dbName":"default","tableName":"jsontable","privileges":null,"tableType":"MANAGED_TABLE","sd":{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsontable","parameters":{},"inputFormat":"org.apache.hadoop.mapred.TextInputFormat","outputFormat":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","compressed":false,"cols":[{"name":"key","type":"int","comment":null,"setName":true,"setType":true,"setComment":false},{"name":"value","type":"string","comment":null,"setName":true,"setType":true,"setComment":false}],"serdeInfo":{"name":null,"parameters":{"serialization.format":"1"},"serializationLib":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","setName":false,"parametersSize":1,"setParameters":true,"setSerializationLib":true},"colsSize":2,"skewedInfo":{"skewedColNames":[],"skewedColValues":[],"skewedColValueLocationMaps":{},"skewedColNamesSize":0,"skewedColNamesIterator":[],"setSkewedColNames":true,"skewedColValuesSize":0,"skewedColValuesIterator":[],"setSkewedColValues":true,"skewedColValueLocationMapsSize":0,"setSkewedColValueLocationMaps":true},"bucketCols":[],"numBuckets":-1,"sortCols":[],"storedAsSubDirectories":false,"setSkewedInfo":true,"parametersSize":0,"setParameters":true,"colsIterator":[{"name":"key","type":"int","comment":null,"setName":true,"setType":true,"setComment":false},{"name":"value","type":"string","comment":null,"setName":true,"setType":true,"setComment":false}],"setCols":true,"setLocation":true,"setInputFormat":true,"setOutputFormat":true,"setCompressed":true,"setNumBuckets":true,"setSerdeInfo":true,"bucketColsSize":0,"bucketColsIterator":[],"setBucketCols":true,"sortColsSize":0,"sortColsIterator":[],"setSortCols":true,"setStoredAsSubDirectories":true},"partitionKeys":[],"viewOriginalText":null,"lastAccessTime":0,"retention":0,"viewExpandedText":null,"partitionKeysSize":0,"setTableType":true,"setTableName":true,"setDbName":true,"setOwner":true,"setCreateTime":true,"setLastAccessTime":true,"setRetention":true,"setSd":true,"partitionKeysIterator":[],"setPartitionKeys":true,"parametersSize":5,"setParameters":true,"setViewOriginalText":false,"setViewExpandedText":false,"setPrivileges":false}} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 b/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b b/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b +++ b/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c b/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c +++ b/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 b/sql/hive/src/test/resources/golden/disable_file_format_check-2-2d27f92dfced693fa3a68ecce5e2e838 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 rename to sql/hive/src/test/resources/golden/disable_file_format_check-2-2d27f92dfced693fa3a68ecce5e2e838 diff --git a/sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 b/sql/hive/src/test/resources/golden/disable_file_format_check-4-a2150709a6ff73326bdf4865dd124a23 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 rename to sql/hive/src/test/resources/golden/disable_file_format_check-4-a2150709a6ff73326bdf4865dd124a23 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b deleted file mode 100644 index 7ae602958428e..0000000000000 --- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b +++ /dev/null @@ -1 +0,0 @@ -238 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b similarity index 100% rename from sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b diff --git a/sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c deleted file mode 100644 index ca21e093aa698..0000000000000 --- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c +++ /dev/null @@ -1 +0,0 @@ -1 {"a1":"b1"} foo1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c new file mode 100644 index 0000000000000..573c4b56de599 --- /dev/null +++ b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c @@ -0,0 +1 @@ +1 {"a1":"b1"} foo1 diff --git a/sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/distinct_stats-0-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/distinct_stats-0-418ec894d08c33fd712eb358f579b7a0 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/distinct_stats-1-10987e425ba8ba8d9c01538f16eab970 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/distinct_stats-1-10987e425ba8ba8d9c01538f16eab970 diff --git a/sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 b/sql/hive/src/test/resources/golden/distinct_stats-2-a2d8f812612283b20ec3f1e92a263440 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 rename to sql/hive/src/test/resources/golden/distinct_stats-2-a2d8f812612283b20ec3f1e92a263440 diff --git a/sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 b/sql/hive/src/test/resources/golden/distinct_stats-3-a4397664f1f109ef0aa5ff36961b25b similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 rename to sql/hive/src/test/resources/golden/distinct_stats-3-a4397664f1f109ef0aa5ff36961b25b diff --git a/sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 b/sql/hive/src/test/resources/golden/distinct_stats-4-e540680af4a857404a0cb7cabc1bcf31 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 rename to sql/hive/src/test/resources/golden/distinct_stats-4-e540680af4a857404a0cb7cabc1bcf31 diff --git a/sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 b/sql/hive/src/test/resources/golden/distinct_stats-5-32e9736bf27c1d2e4399a8125e14befc similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 rename to sql/hive/src/test/resources/golden/distinct_stats-5-32e9736bf27c1d2e4399a8125e14befc diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc b/sql/hive/src/test/resources/golden/distinct_stats-6-297a53801744e6c4786e315e32c6189a similarity index 100% rename from sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc rename to sql/hive/src/test/resources/golden/distinct_stats-6-297a53801744e6c4786e315e32c6189a diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 b/sql/hive/src/test/resources/golden/distinct_stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d similarity index 100% rename from sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 rename to sql/hive/src/test/resources/golden/distinct_stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 b/sql/hive/src/test/resources/golden/distinct_stats-8-ca16024e6f5399b1d035f5b9fd665163 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 rename to sql/hive/src/test/resources/golden/distinct_stats-8-ca16024e6f5399b1d035f5b9fd665163 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/distinct_stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/distinct_stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8 diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee similarity index 100% rename from sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101 diff --git a/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 +++ b/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 +++ b/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-0-a99af48bbcbaba062e9bc387ae2b4975 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-0-a99af48bbcbaba062e9bc387ae2b4975 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-1-e67a0f7ff61a97b2c49386890ea88c54 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-1-e67a0f7ff61a97b2c49386890ea88c54 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-2-19915227905aab376d918b3cada85c25 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-2-19915227905aab376d918b3cada85c25 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-3-e32b952789a115ff02201dfa618d92b2 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-3-e32b952789a115ff02201dfa618d92b2 diff --git a/sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-4-58aff7fa05fba3a7549629a17e285036 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-4-58aff7fa05fba3a7549629a17e285036 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-5-e32b952789a115ff02201dfa618d92b2 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-5-e32b952789a115ff02201dfa618d92b2 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-6-4642f8a18bf7409849f2e91d7a05f352 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-6-4642f8a18bf7409849f2e91d7a05f352 diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 index c9d701778f9ab..2895d472ca5d9 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 @@ -1 +1 @@ -b=2/c=2 \ No newline at end of file +b=2/c=2 diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 index 53e09b6e34202..0f9c2f1d90639 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 @@ -1,3 +1,3 @@ b=1/c=1 b=1/c=2 -b=2/c=2 \ No newline at end of file +b=2/c=2 diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 b/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 index 31b543e8b4122..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 @@ -1,12 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ALTERTABLE_DROPPARTS mp (TOK_PARTSPEC (TOK_PARTVAL b = '1'))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Drop Table Operator: - Drop Table - table: mp - diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 index c9d701778f9ab..2895d472ca5d9 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 @@ -1 +1 @@ -b=2/c=2 \ No newline at end of file +b=2/c=2 diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 b/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 index d1e9fd1cd0a21..30eb53bdc8e70 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -1,10 +1,10 @@ -a string None -b int None -c string None -d string None +a string +b int +c string +d string # Partition Information # col_name data_type comment -c string None -d string None \ No newline at end of file +c string +d string diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 index 3377ef5cf498f..5e6d96c4e60aa 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 @@ -6,4 +6,4 @@ c=India/d=3 c=Russia/d=3 c=US/d=1 c=US/d=2 -c=Uganda/d=2 \ No newline at end of file +c=Uganda/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 index 40e71fb79ad0f..e91541d1527d7 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 @@ -5,4 +5,4 @@ c=Greece/d=2 c=India/d=3 c=Russia/d=3 c=US/d=2 -c=Uganda/d=2 \ No newline at end of file +c=Uganda/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 index c03d86a551c29..316e63e21e7b1 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 @@ -3,4 +3,4 @@ c=France/d=4 c=Germany/d=2 c=Greece/d=2 c=India/d=3 -c=Russia/d=3 \ No newline at end of file +c=Russia/d=3 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 index 133c0256f898f..231c59f365307 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 @@ -2,4 +2,4 @@ c=Canada/d=3 c=France/d=4 c=Germany/d=2 c=Greece/d=2 -c=India/d=3 \ No newline at end of file +c=India/d=3 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 index 98e52eedc2b03..02a7003ca12de 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 @@ -1,3 +1,3 @@ c=Canada/d=3 c=France/d=4 -c=Germany/d=2 \ No newline at end of file +c=Germany/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 index 687f7dd7e13a0..2ffea3b50a9b4 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 @@ -1 +1 @@ -c=France/d=4 \ No newline at end of file +c=France/d=4 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 b/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 index 687f7dd7e13a0..2ffea3b50a9b4 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 @@ -1 +1 @@ -c=France/d=4 \ No newline at end of file +c=France/d=4 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 index 1329d173d6a21..ca4194fbcf3ff 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -1,10 +1,10 @@ -a string None -b int None -c int None -d int None +a string +b int +c int +d int # Partition Information # col_name data_type comment -c int None -d int None \ No newline at end of file +c int +d int diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 index 08051a26d24cc..7ace4dc662306 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 @@ -2,4 +2,4 @@ c=1/d=2 c=2/d=1 c=2/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=30/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 index 7f6e4ae8abf83..1ca1833c09245 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 @@ -1,3 +1,3 @@ c=1/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=30/d=2 diff --git a/sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-13-8117981303487dc4c4873356931ef26a similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-13-8117981303487dc4c4873356931ef26a diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..fee1b1ad01412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1 @@ +c=30/d=2 diff --git a/sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba b/sql/hive/src/test/resources/golden/drop_partitions_filter2-15-67d75c72ea2d3982c55f3a850d93f83c similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-15-67d75c72ea2d3982c55f3a850d93f83c diff --git a/sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-7-f34625fd49a5e655cba3abb5cb8c5417 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318 rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-7-f34625fd49a5e655cba3abb5cb8c5417 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 index 226ef460b53a6..b77f18cd02020 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 @@ -3,4 +3,4 @@ c=1/d=2 c=2/d=1 c=2/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=30/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 index 01562f65d807f..77bc36b96870b 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -1,10 +1,10 @@ -a string None -b int None -c string None -d int None +a string +b int +c string +d int # Partition Information # col_name data_type comment -c string None -d int None \ No newline at end of file +c string +d int diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 index 08051a26d24cc..50e8df00f1597 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 @@ -2,4 +2,4 @@ c=1/d=2 c=2/d=1 c=2/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=3/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 index 7f6e4ae8abf83..20bc2b0c74d32 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 @@ -1,3 +1,3 @@ c=1/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=3/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 index 226ef460b53a6..6200b3ad2dd94 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 @@ -3,4 +3,4 @@ c=1/d=2 c=2/d=1 c=2/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=3/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 index 8caab1c99b27d..73f873dbcf00b 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 +++ b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 @@ -1,10 +1,10 @@ -c1 string None -c2 string None -p string None +c1 string +c2 string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl_protectmode_no_drop, createTime:1389728724, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:c1, type:string, comment:null), FieldSchema(name:c2, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/tbl_protectmode_no_drop/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1389728724, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1389728724}) \ No newline at end of file +Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl_protectmode_no_drop, createTime:1413879999, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:c1, type:string, comment:null), FieldSchema(name:c2, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl_protectmode_no_drop/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413879999, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1413879999, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 +++ b/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 b/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 +++ b/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 b/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 index fbe12dca4efc8..a4b5a45443235 100644 --- a/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 +++ b/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 @@ -1,3 +1,3 @@ p=p1 p=p2 -p=p3 \ No newline at end of file +p=p3 diff --git a/sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3 diff --git a/sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff similarity index 100% rename from sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04 rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff diff --git a/sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7 similarity index 100% rename from sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299 diff --git a/sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208 similarity index 100% rename from sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d diff --git a/sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-3cd14bc5b126ff8b337c4abc09134260 similarity index 100% rename from sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-3cd14bc5b126ff8b337c4abc09134260 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe +++ b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-ce929ee6a92b81d8080ca322c1c38a4b similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-ce929ee6a92b81d8080ca322c1c38a4b diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-f3a5a998099b756a21cf9122a15b09d5 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-f3a5a998099b756a21cf9122a15b09d5 diff --git a/sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49 diff --git a/sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-fed732d2e7d94a4fc02e7694f9f9a39c similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-fed732d2e7d94a4fc02e7694f9f9a39c diff --git a/sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-adab6a0187003ab7ee6f217c9e409d91 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-adab6a0187003ab7ee6f217c9e409d91 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 +++ b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-adab6a0187003ab7ee6f217c9e409d91 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-adab6a0187003ab7ee6f217c9e409d91 diff --git a/sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-4b099f87e221b8fd5c0d0d4a97c0d146 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-4b099f87e221b8fd5c0d0d4a97c0d146 diff --git a/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf b/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf deleted file mode 100644 index 8273b7ed19da6..0000000000000 --- a/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf +++ /dev/null @@ -1,10 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -10 val_10 -100 val_100 -100 val_100 -103 val_103 -103 val_103 -104 val_104 -104 val_104 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 b/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 deleted file mode 100644 index 045906a29a1da..0000000000000 --- a/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 +++ /dev/null @@ -1,10 +0,0 @@ -98 val_98 -98 val_98 -97 val_97 -97 val_97 -96 val_96 -95 val_95 -95 val_95 -92 val_92 -90 val_90 -90 val_90 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 b/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 deleted file mode 100644 index a949a93dfcca6..0000000000000 --- a/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 +++ /dev/null @@ -1 +0,0 @@ -128 diff --git a/sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 b/sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 deleted file mode 100644 index e2799aa7e14b5885aad48117308f489740683449..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 252 zcmV~$15yM4006)yw>8yeIstVOFf?K*Vo(yd3YKK%v^3K=qN#Hcaj zCQO<#ZN{uQ^A;>xvTVhwHS0EP+OloOu08t>96ECB#HlmqE?l~D?Z&M;_Z~cY^6bT{ OH}5`t`tt3^uRs6QQ6wq= diff --git a/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 b/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 deleted file mode 100644 index a949a93dfcca6..0000000000000 --- a/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 +++ /dev/null @@ -1 +0,0 @@ -128 diff --git a/sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 b/sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 deleted file mode 100644 index e2799aa7e14b5885aad48117308f489740683449..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 252 zcmV~$15yM4006)yw>8yeIstVOFf?K*Vo(yd3YKK%v^3K=qN#Hcaj zCQO<#ZN{uQ^A;>xvTVhwHS0EP+OloOu08t>96ECB#HlmqE?l~D?Z&M;_Z~cY^6bT{ OH}5`t`tt3^uRs6QQ6wq= diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d b/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 b/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 b/sql/hive/src/test/resources/golden/explain_rearrange-0-6f7c8515c354fb050829ebd66413425 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 rename to sql/hive/src/test/resources/golden/explain_rearrange-0-6f7c8515c354fb050829ebd66413425 diff --git a/sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 b/sql/hive/src/test/resources/golden/explain_rearrange-1-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 rename to sql/hive/src/test/resources/golden/explain_rearrange-1-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e b/sql/hive/src/test/resources/golden/explain_rearrange-10-3f2680208772a0e51aefc4ef5604dddf similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e rename to sql/hive/src/test/resources/golden/explain_rearrange-10-3f2680208772a0e51aefc4ef5604dddf diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-11-f2ca12a948fd9b5b842168e7c7d7b768 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/explain_rearrange-11-f2ca12a948fd9b5b842168e7c7d7b768 diff --git a/sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee b/sql/hive/src/test/resources/golden/explain_rearrange-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee rename to sql/hive/src/test/resources/golden/explain_rearrange-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/explain_rearrange-13-5baad22ed7efa18d73eb8349e57cf331 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/explain_rearrange-13-5baad22ed7efa18d73eb8349e57cf331 diff --git a/sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef b/sql/hive/src/test/resources/golden/explain_rearrange-14-490d6253b73064ce403e4d04a8bc18f3 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef rename to sql/hive/src/test/resources/golden/explain_rearrange-14-490d6253b73064ce403e4d04a8bc18f3 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-2-24ca942f094b14b92086305cc125e833 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/explain_rearrange-2-24ca942f094b14b92086305cc125e833 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-3-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/explain_rearrange-3-3b0f76816be2c1b18a2058027a19bc9f diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 b/sql/hive/src/test/resources/golden/explain_rearrange-4-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 rename to sql/hive/src/test/resources/golden/explain_rearrange-4-86473a0498e4361e4db0b4a22f2e8571 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-5-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/explain_rearrange-5-d0ec6d66ff349db09fd455eec149efdb diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-6-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/explain_rearrange-6-cda81d86d127fca0e2fbc2161e91400d diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/explain_rearrange-7-5b13cfa4b730e38ef2794c1532968e04 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/explain_rearrange-7-5b13cfa4b730e38ef2794c1532968e04 diff --git a/sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 b/sql/hive/src/test/resources/golden/explain_rearrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 rename to sql/hive/src/test/resources/golden/explain_rearrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-9-73b9ac83dbc9874dc9379ad4364d40ac similarity index 100% rename from sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/explain_rearrange-9-73b9ac83dbc9874dc9379ad4364d40ac diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 b/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 +++ b/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 b/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 index fb12634ea81c1..8059361d2485f 100644 --- a/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 +++ b/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 @@ -1 +1 @@ -500 130091 260.182 0 498 142.92680950752384 20428.072876000006 \ No newline at end of file +500 130091 260.182 0 498 142.92680950752384 20428.072876000006 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 +++ b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 b/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 index d23e05acf7ba5..e34118512c1d7 100644 --- a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 +++ b/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 @@ -497,4 +497,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 index 6280b32facd66..1cf9f21c9ed88 100644 --- a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 +++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 @@ -1,4 +1,4 @@ -key int None -value string None +key int +value string -Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823397, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823397}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1413880056, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413880056}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 index e793ec2f946e5..62d6734063fdd 100644 --- a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 +++ b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 @@ -1,4 +1,4 @@ -key int None -value string None +key int +value string -Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823407}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1413880064, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413880064}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e b/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e index cc545367b951b..31956a614026d 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e @@ -9,4 +9,4 @@ 66 val_66 213 val_213 146 val_146 -406 val_406 \ No newline at end of file +406 val_406 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd b/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd index 140c6590a27fc..51eb2d30c97eb 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd @@ -1 +1 @@ -4 5 0 2010-04-17 \ No newline at end of file +4 5 0 2010-04-17 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 index 51f5701ceae46..7dc9c13cfee04 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 @@ -1 +1 @@ -4 1 1 8 4 5 1 0 9 U 2 2 0 2 1 1 J C A U 2 s 2 NULL NULL NULL NULL NULL NULL 1 j S 6 NULL 1 2 J g 1 e 2 1 2 U P p 3 0 0 0 1 1 1 0 0 0 6 2 j NULL NULL NULL NULL NULL NULL 5 NULL NULL j 2 2 1 2 2 1 1 1 1 1 1 1 1 32 NULL 2010-04-17 \ No newline at end of file +4 1 1 8 4 5 1 0 9 U 2 2 0 2 1 1 J C A U 2 s 2 NULL NULL NULL NULL NULL NULL 1 j S 6 NULL 1 2 J g 1 e 2 1 2 U P p 3 0 0 0 1 1 1 0 0 0 6 2 j NULL NULL NULL NULL NULL NULL 5 NULL NULL j 2 2 1 2 2 1 1 1 1 1 1 1 1 32 NULL 2010-04-17 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 index 69dcc336a73f2..19a6df9f78cd7 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 @@ -1 +1 @@ -5 5 4 \ No newline at end of file +5 5 4 diff --git a/sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-4-fe8b55e4e4098d7a2662338783a50306 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-4-fe8b55e4e4098d7a2662338783a50306 diff --git a/sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-5-cf724251613216ec54f8ac2e6b9b92fd similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-5-cf724251613216ec54f8ac2e6b9b92fd diff --git a/sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d b/sql/hive/src/test/resources/golden/filter_join_breaktask2-6-8c782ae8f8245bdbe90d068a6b577d1e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-6-8c782ae8f8245bdbe90d068a6b577d1e diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e b/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e index c0f577c3cd2fd..d6961f6e46e93 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e @@ -1 +1 @@ -5 name NULL 2 kavin NULL 9 c 8 0 0 7 1 2 0 3 2 NULL 1 NULL 3 2 0 0 5 10 2010-04-17 \ No newline at end of file +5 name NULL 2 kavin NULL 9 c 8 0 0 7 1 2 0 3 2 NULL 1 NULL 3 2 0 0 5 10 2010-04-17 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 index e426b4879bcb6..be23778bd2cc4 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 @@ -1 +1 @@ -5 1 1 1 0 0 4 2010-04-17 \ No newline at end of file +5 1 1 1 0 0 4 2010-04-17 diff --git a/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d +++ b/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 b/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 +++ b/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 b/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 +++ b/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd b/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd index 8939d9f8af186..07df09912b06e 100644 --- a/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd +++ b/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd @@ -306,4 +306,4 @@ val_92 1 1 111 val_95 2 1 111 val_96 1 1 111 val_97 2 1 111 -val_98 2 1 111 \ No newline at end of file +val_98 2 1 111 diff --git a/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d b/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d index 82a0329b4d459..efbfb713807a0 100644 --- a/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d +++ b/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d @@ -306,4 +306,4 @@ 95 2 1 111 96 1 1 111 97 2 1 111 -98 2 1 111 \ No newline at end of file +98 2 1 111 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/groupby12-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/groupby12-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e b/sql/hive/src/test/resources/golden/groupby12-1-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e rename to sql/hive/src/test/resources/golden/groupby12-1-13ab74a58da514fe01dbeda0c3e79883 diff --git a/sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c b/sql/hive/src/test/resources/golden/groupby12-2-fd150794945892f3c926a1881cd819f4 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c rename to sql/hive/src/test/resources/golden/groupby12-2-fd150794945892f3c926a1881cd819f4 diff --git a/sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 b/sql/hive/src/test/resources/golden/groupby12-3-8018bb917a0706925c14421ec2761663 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 rename to sql/hive/src/test/resources/golden/groupby12-3-8018bb917a0706925c14421ec2761663 diff --git a/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..98f8836673e8a --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,309 @@ +3 1 +1 1 +1 1 +3 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +2 1 +1 1 +1 1 +2 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +3 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +2 1 +1 1 +3 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +1 1 +1 1 +3 1 +1 1 +2 1 +1 1 +2 1 +2 1 +2 1 +2 1 +2 1 +1 1 +1 1 +2 1 +1 1 +1 1 +2 1 +3 1 +2 1 +2 1 +1 1 +3 1 +2 1 +1 1 +1 1 +2 1 +1 1 +2 1 +4 1 +1 1 +1 1 +2 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +3 1 +1 1 +4 1 +1 1 +2 1 +2 1 +2 1 +2 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +3 1 +1 1 +1 1 +2 1 +1 1 +3 1 +1 1 +2 1 +1 1 +2 1 +3 1 +2 1 +1 1 +1 1 +2 1 +2 1 +2 1 +3 1 +2 1 +2 1 +1 1 +2 1 +2 1 +1 1 +2 1 +2 1 +1 1 +2 1 +2 1 +1 1 +1 1 +2 1 +5 1 +2 1 +1 1 +2 1 +2 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +1 1 +2 1 +3 1 +1 1 +1 1 +4 1 +2 1 +2 1 +2 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +3 1 +1 1 +1 1 +1 1 +2 1 +1 1 +2 1 +1 1 +3 1 +1 1 +3 1 +2 1 +3 1 +2 1 +2 1 +1 1 +2 1 +3 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +5 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +1 1 +3 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +3 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +3 1 +2 1 +2 1 +1 1 +5 1 +1 1 +3 1 +2 1 +4 1 +1 1 +3 1 +1 1 +2 1 +2 1 +3 1 +1 1 +1 1 +1 1 +2 1 +1 1 +2 1 +3 1 +3 1 +1 1 +1 1 +1 1 +1 1 +3 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +3 1 +1 1 +1 1 +2 1 +2 1 +1 1 +2 1 +2 1 +3 1 +1 1 +4 1 +5 1 +1 1 +1 1 +1 1 +1 1 +2 1 +1 1 +3 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +4 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +3 1 diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e b/sql/hive/src/test/resources/golden/groupby1_limit-3-607512a8dd16cd9ddde561eeabfa51db similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e rename to sql/hive/src/test/resources/golden/groupby1_limit-3-607512a8dd16cd9ddde561eeabfa51db diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 b/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 index 9cf9606d751ac..f92f60d11d4ef 100644 --- a/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 +++ b/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 @@ -2,4 +2,4 @@ 2 2.0 4 4.0 5 15.0 -8 8.0 \ No newline at end of file +8 8.0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 b/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb b/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c index 3210a3048ff48..ba568b8fd6cf2 100644 --- a/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c +++ b/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c @@ -7,4 +7,4 @@ 6 5 6398.0 7 6 7735.0 8 8 8762.0 -9 7 91047.0 \ No newline at end of file +9 7 91047.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e b/sql/hive/src/test/resources/golden/groupby2_limit-1-a56d6499aef913e11ef599ac8b4f2a25 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e rename to sql/hive/src/test/resources/golden/groupby2_limit-1-a56d6499aef913e11ef599ac8b4f2a25 diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 b/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 deleted file mode 100644 index 9cf9606d751ac..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 +++ /dev/null @@ -1,5 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a b/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a new file mode 100644 index 0000000000000..f92f60d11d4ef --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a @@ -0,0 +1,5 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 3210a3048ff48..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 -1 71 116414.0 -2 69 225571.0 -3 62 332004.0 -4 74 452763.0 -5 6 5397.0 -6 5 6398.0 -7 6 7735.0 -8 8 8762.0 -9 7 91047.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..ba568b8fd6cf2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,10 @@ +0 1 00.0 +1 71 116414.0 +2 69 225571.0 +3 62 332004.0 +4 74 452763.0 +5 6 5397.0 +6 5 6398.0 +7 6 7735.0 +8 8 8762.0 +9 7 91047.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 922f4bfc83e44..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f index 3210a3048ff48..ba568b8fd6cf2 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ 6 5 6398.0 7 6 7735.0 8 8 8762.0 -9 7 91047.0 \ No newline at end of file +9 7 91047.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 b/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 new file mode 100644 index 0000000000000..ba568b8fd6cf2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 @@ -0,0 +1,10 @@ +0 1 00.0 +1 71 116414.0 +2 69 225571.0 +3 62 332004.0 +4 74 452763.0 +5 6 5397.0 +6 5 6398.0 +7 6 7735.0 +8 8 8762.0 +9 7 91047.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c deleted file mode 100644 index 3210a3048ff48..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 -1 71 116414.0 -2 69 225571.0 -3 62 332004.0 -4 74 452763.0 -5 6 5397.0 -6 5 6398.0 -7 6 7735.0 -8 8 8762.0 -9 7 91047.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c deleted file mode 100644 index 922f4bfc83e44..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f index f55b5c9eef39f..8b1acc12b635c 100644 --- a/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ 6 7 8 -9 \ No newline at end of file +9 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f new file mode 100644 index 0000000000000..8b1acc12b635c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f55b5c9eef39f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f index 349d8b75d942b..3975bfc1af512 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -1 +1 @@ -130091 \ No newline at end of file +130091 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f index 349d8b75d942b..3975bfc1af512 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -1 +1 @@ -130091 \ No newline at end of file +130091 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f index f55b5c9eef39f..8b1acc12b635c 100644 --- a/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ 6 7 8 -9 \ No newline at end of file +9 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f new file mode 100644 index 0000000000000..8b1acc12b635c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f55b5c9eef39f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f new file mode 100644 index 0000000000000..8b1acc12b635c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f55b5c9eef39f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f new file mode 100644 index 0000000000000..8b1acc12b635c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f55b5c9eef39f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e +++ b/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 index dfca4e169cbe8..951e74db0fe23 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 @@ -7,4 +7,4 @@ 10 10.0 11 11.0 12 24.0 -15 30.0 \ No newline at end of file +15 30.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-ab2390a3b0ba76907a7bee390b2924a1 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 rename to sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-ab2390a3b0ba76907a7bee390b2924a1 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 index dfca4e169cbe8..951e74db0fe23 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 @@ -7,4 +7,4 @@ 10 10.0 11 11.0 12 24.0 -15 30.0 \ No newline at end of file +15 30.0 diff --git a/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e +++ b/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e +++ b/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 index 0e40f3f1a49a9..c156bd5f5d8c6 100644 --- a/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 val_495 1 496 val_496 1 497 val_497 1 -498 val_498 1 \ No newline at end of file +498 val_498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 index df07a9da29f01..93e965c771403 100644 --- a/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 3 \ No newline at end of file +498 3 diff --git a/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 index cf0c065dc6532..742479ac713c5 100644 --- a/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 val_495 1 496 val_496 1 497 val_497 1 -498 val_498 3 \ No newline at end of file +498 val_498 3 diff --git a/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 index 4ef88d57aa72b..deaf15e462910 100644 --- a/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 95 val_95 1 96 val_96 1 97 val_97 1 -98 val_98 1 \ No newline at end of file +98 val_98 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 index 0e40f3f1a49a9..c156bd5f5d8c6 100644 --- a/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 val_495 1 496 val_496 1 497 val_497 1 -498 val_498 1 \ No newline at end of file +498 val_498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 index 4ef88d57aa72b..deaf15e462910 100644 --- a/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 95 val_95 1 96 val_96 1 97 val_97 1 -98 val_98 1 \ No newline at end of file +98 val_98 1 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index ae9bbc3e2c2f7..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 -1 71 132828.0 -2 69 251142.0 -3 62 364008.0 -4 74 4105526.0 -5 6 5794.0 -6 5 6796.0 -7 6 71470.0 -8 8 81524.0 -9 7 92094.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..1b9d97300aa08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,10 @@ +0 1 00.0 +1 71 132828.0 +2 69 251142.0 +3 62 364008.0 +4 74 4105526.0 +5 6 5794.0 +6 5 6796.0 +7 6 71470.0 +8 8 81524.0 +9 7 92094.0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 64bb7c62c1885..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 1 -1 71 132828.0 10044 71 -2 69 251142.0 15780 69 -3 62 364008.0 20119 62 -4 74 4105526.0 30965 74 -5 6 5794.0 278 6 -6 5 6796.0 331 5 -7 6 71470.0 447 6 -8 8 81524.0 595 8 -9 7 92094.0 577 7 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 index 6b6a788e382db..0b7e79a79bd11 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 @@ -3,4 +3,4 @@ 4 1 5 1 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 index c33eed60c8d5b..d63ed5dbe78bb 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 @@ -3,4 +3,4 @@ 8 1 10 1 16 1 -18 1 \ No newline at end of file +18 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 index 6b6a788e382db..0b7e79a79bd11 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 @@ -3,4 +3,4 @@ 4 1 5 1 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 index c33eed60c8d5b..d63ed5dbe78bb 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 @@ -3,4 +3,4 @@ 8 1 10 1 16 1 -18 1 \ No newline at end of file +18 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd deleted file mode 100644 index 922f4bfc83e44..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e deleted file mode 100644 index 106132fc993d3..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e +++ /dev/null @@ -1,5 +0,0 @@ -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 deleted file mode 100644 index 65235356ea425..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 +++ /dev/null @@ -1,5 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd deleted file mode 100644 index 922f4bfc83e44..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 deleted file mode 100644 index 017878bc9bee5..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 4 1878.0 878 6 -1 5 1729.0 729 8 -1 6 11282.0 1282 12 -1 6 11494.0 1494 11 -1 7 11171.0 1171 11 -1 7 11516.0 1516 10 -1 8 11263.0 1263 10 -1 9 12294.0 2294 14 -1 9 12654.0 2654 16 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae deleted file mode 100644 index f21a658e3c68f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae +++ /dev/null @@ -1,32 +0,0 @@ -5 1 5102.0 102 2 -5 1 5116.0 116 2 -5 1 515.0 15 3 -5 1 553.0 53 1 -5 1 554.0 54 1 -5 1 557.0 57 1 -6 1 6134.0 134 2 -6 1 664.0 64 1 -6 1 665.0 65 1 -6 1 666.0 66 1 -6 1 669.0 69 1 -7 1 7144.0 144 2 -7 1 7152.0 152 2 -7 1 7210.0 210 3 -7 1 774.0 74 1 -7 1 777.0 77 1 -7 1 778.0 78 1 -8 1 8166.0 166 2 -8 1 8168.0 168 2 -8 1 88.0 8 1 -8 1 880.0 80 1 -8 1 882.0 82 1 -8 1 885.0 85 1 -8 1 886.0 86 1 -8 1 887.0 87 1 -9 1 9190.0 190 2 -9 1 9194.0 194 2 -9 1 9196.0 196 2 -9 1 9270.0 270 3 -9 1 99.0 9 1 -9 1 992.0 92 1 -9 1 996.0 96 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e deleted file mode 100644 index 106132fc993d3..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e +++ /dev/null @@ -1,5 +0,0 @@ -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 deleted file mode 100644 index 65235356ea425..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 +++ /dev/null @@ -1,5 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 index dbc7aebdbbae5..dcb604016a969 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 @@ -2,4 +2,4 @@ 6 5 7 6 8 8 -9 7 \ No newline at end of file +9 7 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 index ebf9d6978dec6..cecfbbd281537 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 @@ -2,4 +2,4 @@ 1 71 115 2 69 111 3 62 99 -4 74 124 \ No newline at end of file +4 74 124 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 index a5ae9e2a62227..83d4ac2489823 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 @@ -1,2 +1,2 @@ 100 2 -200 2 \ No newline at end of file +200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 index 16c7a647a2344..ab65c1a003b5b 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 @@ -1 +1 @@ -400 1 \ No newline at end of file +400 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 index 67f077e51a647..11c303a71007c 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 @@ -1,2 +1,2 @@ val_100 2 -val_200 2 \ No newline at end of file +val_200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 index b06ad20135fbe..1e4c20551b68b 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 @@ -1 +1 @@ -val_200 2 \ No newline at end of file +val_200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 index a5ae9e2a62227..83d4ac2489823 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 @@ -1,2 +1,2 @@ 100 2 -200 2 \ No newline at end of file +200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 index 16c7a647a2344..ab65c1a003b5b 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 @@ -1 +1 @@ -400 1 \ No newline at end of file +400 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 index 67f077e51a647..11c303a71007c 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 @@ -1,2 +1,2 @@ val_100 2 -val_200 2 \ No newline at end of file +val_200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 index b06ad20135fbe..1e4c20551b68b 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 @@ -1 +1 @@ -val_200 2 \ No newline at end of file +val_200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb b/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb index efe5cc7795c65..26dbfc34ad4fb 100644 --- a/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb +++ b/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb @@ -1 +1 @@ --30.33 \ No newline at end of file +-30.33 diff --git a/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 b/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 index efe5cc7795c65..26dbfc34ad4fb 100644 --- a/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 +++ b/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 @@ -1 +1 @@ --30.33 \ No newline at end of file +-30.33 diff --git a/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 b/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 b/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 deleted file mode 100644 index f2a91fe3bfab0..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 +++ /dev/null @@ -1,13 +0,0 @@ -0 val_0 1 -2 val_2 1 -4 val_4 1 -5 val_5 1 -8 val_8 1 -9 val_9 1 -10 val_10 1 -11 val_11 1 -12 val_12 1 -15 val_15 1 -17 val_17 1 -18 val_18 1 -19 val_19 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f b/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f deleted file mode 100644 index c8e666cb01e8e..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f +++ /dev/null @@ -1,14 +0,0 @@ -0 3 -5 3 -12 2 -15 2 -18 2 -2 1 -4 1 -8 1 -9 1 -10 1 -11 1 -17 1 -19 1 -20 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb b/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb deleted file mode 100644 index 96a824a81c589..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb +++ /dev/null @@ -1,3 +0,0 @@ -19 val_19 19 val_19 -18 val_18 18 val_18 -17 val_17 17 val_17 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d b/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d deleted file mode 100644 index a79396dac079a..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d +++ /dev/null @@ -1,13 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 b/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 deleted file mode 100644 index f2a91fe3bfab0..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 +++ /dev/null @@ -1,13 +0,0 @@ -0 val_0 1 -2 val_2 1 -4 val_4 1 -5 val_5 1 -8 val_8 1 -9 val_9 1 -10 val_10 1 -11 val_11 1 -12 val_12 1 -15 val_15 1 -17 val_17 1 -18 val_18 1 -19 val_19 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d b/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d deleted file mode 100644 index a79396dac079a..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d +++ /dev/null @@ -1,13 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f index ae9bbc3e2c2f7..1b9d97300aa08 100644 --- a/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ 6 5 6796.0 7 6 71470.0 8 8 81524.0 -9 7 92094.0 \ No newline at end of file +9 7 92094.0 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 64bb7c62c1885..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 1 -1 71 132828.0 10044 71 -2 69 251142.0 15780 69 -3 62 364008.0 20119 62 -4 74 4105526.0 30965 74 -5 6 5794.0 278 6 -6 5 6796.0 331 5 -7 6 71470.0 447 6 -8 8 81524.0 595 8 -9 7 92094.0 577 7 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 b/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 deleted file mode 100644 index 10f4a1f5ff34c..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 +++ /dev/null @@ -1,6 +0,0 @@ -1 11 1 -2 12 1 -3 13 1 -7 17 1 -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 deleted file mode 100644 index c1cc4ee204773..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 +++ /dev/null @@ -1,5 +0,0 @@ -1 1 1 -1 2 1 -1 3 1 -1 7 1 -1 8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 deleted file mode 100644 index f0192040e147b..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 +++ /dev/null @@ -1,5 +0,0 @@ -1 2 1 -2 3 1 -3 4 1 -7 8 1 -8 9 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index b6c2eb98e5e49..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -2 1 -4 1 -6 1 -14 1 -16 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 8e7ee8a2b47bb..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -1 1 -2 1 -2 1 -3 1 -3 1 -7 1 -7 1 -8 2 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 0b2b54cd94e4f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -2 1 -2 1 -3 1 -4 1 -6 1 -7 1 -8 2 -14 1 -16 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 3d4708b7c9d64..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 2 -2 2 -3 2 -7 2 -8 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 b/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 deleted file mode 100644 index 7ca6b0b28a960..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 2 1 -2 1 12 2 1 -3 1 13 2 1 -7 1 17 2 1 -8 1 18 2 1 -8 1 28 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 58e16ef3c0ef3..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 2 11 1 -2 2 12 1 -3 2 13 1 -7 2 17 1 -8 2 18 1 -8 2 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd b/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd deleted file mode 100644 index 10f4a1f5ff34c..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd +++ /dev/null @@ -1,6 +0,0 @@ -1 11 1 -2 12 1 -3 13 1 -7 17 1 -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 deleted file mode 100644 index d15db8c5d079f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 +++ /dev/null @@ -1 +0,0 @@ -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 b/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 deleted file mode 100644 index 3d1609d961673..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 +++ /dev/null @@ -1,2 +0,0 @@ -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 b/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 index c6bb9dbfd6497..bfca78293c988 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 @@ -1,2 +1,2 @@ 0 -11 \ No newline at end of file +11 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 b/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 index c6bb9dbfd6497..bfca78293c988 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 @@ -1,2 +1,2 @@ 0 -11 \ No newline at end of file +11 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 b/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 deleted file mode 100644 index ded2854cdf564..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 +++ /dev/null @@ -1,6 +0,0 @@ -1 3 3 0.0 -1 1 1 2.0 -1 1 1 4.0 -1 3 3 5.0 -1 1 1 8.0 -1 1 1 9.0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 b/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 deleted file mode 100644 index 487b4c4a5cc6f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 +++ /dev/null @@ -1,6 +0,0 @@ -0 1 3 3 0.0 -2 1 1 1 2.0 -4 1 1 1 4.0 -5 1 3 3 5.0 -8 1 1 1 8.0 -9 1 1 1 9.0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 b/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 b/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 deleted file mode 100644 index 6a5fe2835fc56..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 +++ /dev/null @@ -1 +0,0 @@ -6 10 10 28.0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 b/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 index c5b99ed941efc..e6a233467dcf5 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 @@ -3,4 +3,4 @@ 13 1 17 1 18 1 -28 1 \ No newline at end of file +28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 b/sql/hive/src/test/resources/golden/groupby_sort_2-5-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 rename to sql/hive/src/test/resources/golden/groupby_sort_2-5-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f b/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f index 4e31460a412ba..e7273779ac1b8 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f @@ -2,4 +2,4 @@ 2 1 3 1 7 1 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce b/sql/hive/src/test/resources/golden/groupby_sort_3-5-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce rename to sql/hive/src/test/resources/golden/groupby_sort_3-5-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd index 4e31460a412ba..e7273779ac1b8 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd @@ -2,4 +2,4 @@ 2 1 3 1 7 1 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 b/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c b/sql/hive/src/test/resources/golden/groupby_sort_4-5-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c rename to sql/hive/src/test/resources/golden/groupby_sort_4-5-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 b/sql/hive/src/test/resources/golden/groupby_sort_5-13-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 rename to sql/hive/src/test/resources/golden/groupby_sort_5-13-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb b/sql/hive/src/test/resources/golden/groupby_sort_5-20-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb rename to sql/hive/src/test/resources/golden/groupby_sort_5-20-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f b/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f index 4e31460a412ba..e7273779ac1b8 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f @@ -2,4 +2,4 @@ 2 1 3 1 7 1 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 b/sql/hive/src/test/resources/golden/groupby_sort_5-5-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 rename to sql/hive/src/test/resources/golden/groupby_sort_5-5-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd index 4e31460a412ba..e7273779ac1b8 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd @@ -2,4 +2,4 @@ 2 1 3 1 7 1 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 b/sql/hive/src/test/resources/golden/groupby_sort_6-9-591e03d1cfc10821a601498df1ed6675 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 rename to sql/hive/src/test/resources/golden/groupby_sort_6-9-591e03d1cfc10821a601498df1ed6675 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 b/sql/hive/src/test/resources/golden/groupby_sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc similarity index 100% rename from sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 rename to sql/hive/src/test/resources/golden/groupby_sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d index 7813681f5b41c..7ed6ff82de6bc 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d @@ -1 +1 @@ -5 \ No newline at end of file +5 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 b/sql/hive/src/test/resources/golden/groupby_sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc similarity index 100% rename from sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 rename to sql/hive/src/test/resources/golden/groupby_sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d index 7813681f5b41c..7ed6ff82de6bc 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d @@ -1 +1 @@ -5 \ No newline at end of file +5 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a b/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/groupby_sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 rename to sql/hive/src/test/resources/golden/groupby_sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c b/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c index 3d4708b7c9d64..612dcbb640d46 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c @@ -2,4 +2,4 @@ 2 2 3 2 7 2 -8 4 \ No newline at end of file +8 4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 deleted file mode 100644 index 10f4a1f5ff34c..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 +++ /dev/null @@ -1,6 +0,0 @@ -1 11 1 -2 12 1 -3 13 1 -7 17 1 -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 deleted file mode 100644 index c1cc4ee204773..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 +++ /dev/null @@ -1,5 +0,0 @@ -1 1 1 -1 2 1 -1 3 1 -1 7 1 -1 8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 deleted file mode 100644 index f0192040e147b..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 +++ /dev/null @@ -1,5 +0,0 @@ -1 2 1 -2 3 1 -3 4 1 -7 8 1 -8 9 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index b6c2eb98e5e49..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -2 1 -4 1 -6 1 -14 1 -16 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 8e7ee8a2b47bb..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -1 1 -2 1 -2 1 -3 1 -3 1 -7 1 -7 1 -8 2 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 0b2b54cd94e4f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -2 1 -2 1 -3 1 -4 1 -6 1 -7 1 -8 2 -14 1 -16 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 3d4708b7c9d64..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 2 -2 2 -3 2 -7 2 -8 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 deleted file mode 100644 index 7ca6b0b28a960..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 2 1 -2 1 12 2 1 -3 1 13 2 1 -7 1 17 2 1 -8 1 18 2 1 -8 1 28 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 58e16ef3c0ef3..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 2 11 1 -2 2 12 1 -3 2 13 1 -7 2 17 1 -8 2 18 1 -8 2 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd deleted file mode 100644 index 10f4a1f5ff34c..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd +++ /dev/null @@ -1,6 +0,0 @@ -1 11 1 -2 12 1 -3 13 1 -7 17 1 -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 deleted file mode 100644 index d15db8c5d079f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 +++ /dev/null @@ -1 +0,0 @@ -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 deleted file mode 100644 index 3d1609d961673..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 +++ /dev/null @@ -1,2 +0,0 @@ -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a b/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa b/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa rename to sql/hive/src/test/resources/golden/groupby_sort_test_1-6-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f b/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f index a99cee758fe57..3cb614bdd84e8 100644 --- a/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f +++ b/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f @@ -1 +1 @@ -3556498 \ No newline at end of file +3556498 diff --git a/sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf b/sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 b/sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 b/sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 b/sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 b/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 deleted file mode 100644 index d7c6f236687d8..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_2__, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_2__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344545}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 b/sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 b/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 deleted file mode 100644 index 4c6ec0ba34bb8..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:src_idx_src_index_3, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/src_idx_src_index_3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344545}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 b/sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 b/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 deleted file mode 100644 index 0c6af94247b85..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_4__, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_4__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 b/sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d b/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d deleted file mode 100644 index e4a5816e6f1cc..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_5__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_5__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{escape.delim=\, serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad b/sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 b/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 deleted file mode 100644 index 3d8751ca47049..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 +++ /dev/null @@ -1,5 +0,0 @@ -key int from deserializer -_bucketname string from deserializer -_offsets array from deserializer - -Detailed Table Information Table(tableName:default__src_src_index_6__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_6__, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f b/sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 b/sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 b/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 deleted file mode 100644 index 0e4852b319052..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 +++ /dev/null @@ -1,5 +0,0 @@ -key int from deserializer -_bucketname string from deserializer -_offsets array from deserializer - -Detailed Table Information Table(tableName:src_idx_src_index_7, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/src_idx_src_index_7, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e b/sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 deleted file mode 100644 index 41a5492fc5331..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 b/sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf b/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf deleted file mode 100644 index 8212bf8b7d2fb..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_9__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_9__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{prop2=val2, prop1=val1, transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca b/sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 b/sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 b/sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe b/sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f b/sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 b/sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 b/sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 b/sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 b/sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 b/sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b b/sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 b/sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d deleted file mode 100644 index e8310385c56dc..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d +++ /dev/null @@ -1 +0,0 @@ -src \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b b/sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 b/sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c b/sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 b/sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/index_serde-0-db64b724719d27c7f0db4f51f5c4edaa similarity index 100% rename from sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/index_serde-0-db64b724719d27c7f0db4f51f5c4edaa diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 b/sql/hive/src/test/resources/golden/index_serde-1-6560d12b69d55e5297a145ebc4bb0cb3 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 rename to sql/hive/src/test/resources/golden/index_serde-1-6560d12b69d55e5297a145ebc4bb0cb3 diff --git a/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 b/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 deleted file mode 100644 index 48522980f81a8..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 +++ /dev/null @@ -1,3 +0,0 @@ -number int from deserializer -first_name string from deserializer -last_name string from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b b/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b deleted file mode 100644 index 4ed570f9070eb..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b +++ /dev/null @@ -1,5 +0,0 @@ -7 Sylvester McCoy -8 Paul McGann -9 Christopher Eccleston -10 David Tennant -11 Matt Smith \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/index_serde-10-c85e061ea9c5b90ca69b7450faad14b6 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 rename to sql/hive/src/test/resources/golden/index_serde-10-c85e061ea9c5b90ca69b7450faad14b6 diff --git a/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b b/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b new file mode 100644 index 0000000000000..63d56733b58b0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b @@ -0,0 +1,5 @@ +7 Sylvester McCoy +8 Paul McGann +9 Christopher Eccleston +10 David Tennant +11 Matt Smith diff --git a/sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 b/sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad b/sql/hive/src/test/resources/golden/index_serde-12-309e916d683a1a12ab62565697cb0046 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad rename to sql/hive/src/test/resources/golden/index_serde-12-309e916d683a1a12ab62565697cb0046 diff --git a/sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 b/sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 b/sql/hive/src/test/resources/golden/index_serde-13-d590fd7cb9d433143de490d75686dd4 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 rename to sql/hive/src/test/resources/golden/index_serde-13-d590fd7cb9d433143de490d75686dd4 diff --git a/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 b/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 new file mode 100644 index 0000000000000..e716294e919d1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 @@ -0,0 +1,3 @@ +number int from deserializer +first_name string from deserializer +last_name string from deserializer diff --git a/sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c b/sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 b/sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 b/sql/hive/src/test/resources/golden/index_serde-3-ebab588c84a7a29f03b41dcd98132229 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 rename to sql/hive/src/test/resources/golden/index_serde-3-ebab588c84a7a29f03b41dcd98132229 diff --git a/sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/index_serde-4-afcf2a156ccd4f79a0489b4593908d79 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a rename to sql/hive/src/test/resources/golden/index_serde-4-afcf2a156ccd4f79a0489b4593908d79 diff --git a/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 b/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 deleted file mode 100644 index d2ca633d0ae01..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 +++ /dev/null @@ -1,5 +0,0 @@ -number int from deserializer -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__doctors_doctors_index__, dbName:default, owner:null, createTime:1389729651, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:number, type:int, comment:from deserializer), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/default__doctors_doctors_index__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:number, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389729651}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 b/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 new file mode 100644 index 0000000000000..c344129fb8f69 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 @@ -0,0 +1,5 @@ +number int from deserializer +_bucketname string +_offsets array + +Detailed Table Information Table(tableName:default__doctors_doctors_index__, dbName:default, owner:marmbrus, createTime:1414101838, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:number, type:int, comment:from deserializer), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1438070583820061187/default__doctors_doctors_index__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:number, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1414101838}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) diff --git a/sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 b/sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae b/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 b/sql/hive/src/test/resources/golden/index_serde-6-e6ff4b23b7f102e359afb4d53a1dedc3 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 rename to sql/hive/src/test/resources/golden/index_serde-6-e6ff4b23b7f102e359afb4d53a1dedc3 diff --git a/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/index_serde-7-c9d7dcde469d3b9a66965a64dd15e4ae similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/index_serde-7-c9d7dcde469d3b9a66965a64dd15e4ae diff --git a/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 b/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/index_serde-8-3b03210f94ec40db9ab02620645014d1 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/index_serde-8-3b03210f94ec40db9ab02620645014d1 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/index_serde-9-35f48c7d6fa164bb84643657bc9280a8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/index_serde-9-35f48c7d6fa164bb84643657bc9280a8 diff --git a/sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 b/sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e b/sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 b/sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd b/sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 b/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 index 3b7cf42f96358..821c3c8c89252 100644 --- a/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 +++ b/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 @@ -1025,4 +1025,4 @@ 498 val_498 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 b/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 index 010e999c36749..e8a910f80f457 100644 --- a/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 +++ b/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 @@ -1,3 +1,3 @@ -a int None +a int -Detailed Table Information Table(tableName:test, dbName:default, owner:marmbrus, createTime:1389729862, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389729862}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test, dbName:default, owner:marmbrus, createTime:1413881850, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413881850}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 b/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 +++ b/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f b/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f +++ b/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 b/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 index e00bf4ec2c7e5..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 +++ b/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 @@ -1,17 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_DESCTABLE (TOK_TABTYPE TEST1)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Describe Table Operator: - Describe Table - table: TEST1 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 index 743be67e8d1c3..d3ffb995aff4b 100644 --- a/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 +++ b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 @@ -1,2 +1,2 @@ -a int None -b double None \ No newline at end of file +a int +b double diff --git a/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 b/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 index 175d371fd09c9..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 +++ b/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 @@ -1,17 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_DESCTABLE (TOK_TABTYPE TEST10)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Describe Table Operator: - Describe Table - table: TEST10 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 b/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 index 8dcdf43e31be3..4cb356c235573 100644 --- a/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 +++ b/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 @@ -1,10 +1,10 @@ -key int None -value string None -ds string None -hr string None +key int +value string +ds string +hr string # Partition Information # col_name data_type comment -ds string None -hr string None \ No newline at end of file +ds string +hr string diff --git a/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f index 37dd922fd0787..46057aa0a8fca 100644 --- a/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 28 val_28 37 val_37 90 val_90 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 b/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 index fdf93911ee847..d8bd3b13b83f1 100644 --- a/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 +++ b/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 @@ -7,4 +7,4 @@ 66 val_66 82 val_82 86 val_86 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e +++ b/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 +++ b/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c b/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c index 4220cf5f30392..a66a07386eef8 100644 --- a/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c +++ b/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c @@ -308,4 +308,4 @@ 222 2008-04-08 12 403 2008-04-08 12 400 2008-04-08 12 -200 2008-04-08 12 \ No newline at end of file +200 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 +++ b/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f index 37dd922fd0787..46057aa0a8fca 100644 --- a/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 28 val_28 37 val_37 90 val_90 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd b/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd index d6e0c29932b9b..9ee31317478d5 100644 --- a/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd +++ b/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd @@ -102,4 +102,4 @@ 152 val_152 194 val_194 126 val_126 -169 val_169 \ No newline at end of file +169 val_169 diff --git a/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f index 703a4eef24f3f..0190981db84ed 100644 --- a/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 97 val_97 97 val_97 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf b/sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 b/sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 4335dce6a9929..0000000000000 --- a/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -10 val_10 -11 val_11 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 index ded361eb294f0..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 +++ b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 @@ -1,2 +1,2 @@ -key int None -value string None +key int +value string diff --git a/sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 b/sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 b/sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 b/sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 1c48b3680a3ac..0000000000000 --- a/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,11 +0,0 @@ -NULL NULL --1461153966 {"myint":49,"mystring":"343","underscore_int":7} --1952710705 {"myint":25,"mystring":"125","underscore_int":5} --734328905 {"myint":16,"mystring":"64","underscore_int":4} --751827636 {"myint":4,"mystring":"8","underscore_int":2} -1244525196 {"myint":36,"mystring":"216","underscore_int":6} -1638581586 {"myint":64,"mystring":"512","underscore_int":8} -1712634731 {"myint":0,"mystring":"0","underscore_int":0} -336964422 {"myint":81,"mystring":"729","underscore_int":9} -465985201 {"myint":1,"mystring":"1","underscore_int":1} -477111225 {"myint":9,"mystring":"27","underscore_int":3} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc b/sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a b/sql/hive/src/test/resources/golden/input19-1-f32df514de8156b5f5b435eea2c9be40 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a rename to sql/hive/src/test/resources/golden/input19-1-f32df514de8156b5f5b435eea2c9be40 diff --git a/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 b/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 index 795dbe3a976eb..db9438946fc84 100644 --- a/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 +++ b/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 @@ -1 +1 @@ -127.0.0.1 NULL frank 10/Oct/2000:13:55:36 -0700 GET /apache_pb.gif HTTP/1.0 200 2326 \ No newline at end of file +127.0.0.1 NULL frank 10/Oct/2000:13:55:36 -0700 GET /apache_pb.gif HTTP/1.0 200 2326 diff --git a/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 b/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 index fdf93911ee847..d8bd3b13b83f1 100644 --- a/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 +++ b/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 @@ -7,4 +7,4 @@ 66 val_66 82 val_82 86 val_86 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 b/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 index 8a8f1a1b8bbe1..f2f1112224cd7 100644 --- a/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 +++ b/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 @@ -2,4 +2,4 @@ 37 val_37 66 val_66 86 val_86 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 index 743be67e8d1c3..d3ffb995aff4b 100644 --- a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 +++ b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 @@ -1,2 +1,2 @@ -a int None -b double None \ No newline at end of file +a int +b double diff --git a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd index 743be67e8d1c3..d3ffb995aff4b 100644 --- a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd +++ b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd @@ -1,2 +1,2 @@ -a int None -b double None \ No newline at end of file +a int +b double diff --git a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b index ca0726f517eeb..77eaef91c9c3f 100644 --- a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b +++ b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b @@ -1,3 +1,3 @@ -a array None -b double None -c map None \ No newline at end of file +a array +b double +c map diff --git a/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c index 2c861553f9aa2..2dd749277aa48 100644 --- a/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c @@ -1,4 +1,4 @@ src srcpart test2a -test2b \ No newline at end of file +test2b diff --git a/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c index 3e40a0c866d4d..d2cb69524ba34 100644 --- a/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c @@ -1,3 +1,3 @@ src srcpart -test2b \ No newline at end of file +test2b diff --git a/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 b/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 index 5337f342fedd8..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 +++ b/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 @@ -1,17 +0,0 @@ -ABSTRACT SYNTAX TREE: - TOK_SHOWTABLES - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Table Operator: - Show Tables - database name: default - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 b/sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 b/sql/hive/src/test/resources/golden/input21-1-c45ad493e95150b580be778da6065f36 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 rename to sql/hive/src/test/resources/golden/input21-1-c45ad493e95150b580be778da6065f36 diff --git a/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe b/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe index c6c298df200ca..8b39955512bc7 100644 --- a/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe +++ b/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe @@ -7,4 +7,4 @@ NULL 1 same 5 NULL NULL same 6 1.0 NULL same 7 1.0 1 same 8 -1.0 1 same 9 \ No newline at end of file +1.0 1 same 9 diff --git a/sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 b/sql/hive/src/test/resources/golden/input22-1-b663ec84da3f9d9b9594ea2da81b1442 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 rename to sql/hive/src/test/resources/golden/input22-1-b663ec84da3f9d9b9594ea2da81b1442 diff --git a/sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 b/sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 b/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 index 336ba0545635d..891dedb34c1dd 100644 --- a/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 +++ b/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 @@ -7,4 +7,4 @@ 103 103 104 -104 \ No newline at end of file +104 diff --git a/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 b/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 +++ b/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 b/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 index b8fd0ab6545d6..833b21e4d6e4d 100644 --- a/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 +++ b/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 @@ -2,4 +2,4 @@ 0 val_0 2008-04-08 11 0 val_0 2008-04-08 11 2 val_2 2008-04-08 11 -4 val_4 2008-04-08 11 \ No newline at end of file +4 val_4 2008-04-08 11 diff --git a/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 b/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 index 899417ee77ad8..badda49814562 100644 --- a/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 +++ b/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 @@ -2,4 +2,4 @@ 86 val_86 27 val_27 165 val_165 -255 val_255 \ No newline at end of file +255 val_255 diff --git a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d index 743be67e8d1c3..d3ffb995aff4b 100644 --- a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d +++ b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d @@ -1,2 +1,2 @@ -a int None -b double None \ No newline at end of file +a int +b double diff --git a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 b/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 index 594b29ca1410f..bd673a6c1f1d4 100644 --- a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 +++ b/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 @@ -1,4 +1,4 @@ -a array None -b double None -c map None -x double None \ No newline at end of file +a array +b double +c map +x double diff --git a/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c index ac382c7369264..f5b9883df09c0 100644 --- a/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c @@ -1,4 +1,4 @@ src srcpart test3a -test3c \ No newline at end of file +test3c diff --git a/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 b/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 index cb17be511e875..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 +++ b/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ALTERTABLE_REPLACECOLS TEST3c (TOK_TABCOLLIST (TOK_TABCOL R1 TOK_INT) (TOK_TABCOL R2 TOK_DOUBLE))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Alter Table Operator: - Alter Table - type: replace columns - new columns: r1 int, r2 double - old name: TEST3c - diff --git a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 b/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 index b906fd3c2e775..ea55abd792314 100644 --- a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 +++ b/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 @@ -1,4 +1,4 @@ -r1 int None -r2 double None +r1 int +r2 double -Detailed Table Information Table(tableName:test3c, dbName:default, owner:marmbrus, createTime:1389730377, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:r1, type:int, comment:null), FieldSchema(name:r2, type:double, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/test3c, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1389730378, transient_lastDdlTime=1389730378}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test3c, dbName:default, owner:marmbrus, createTime:1413882084, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:r1, type:int, comment:null), FieldSchema(name:r2, type:double, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test3c, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413882084, transient_lastDdlTime=1413882084, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 index ca0726f517eeb..77eaef91c9c3f 100644 --- a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 +++ b/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 @@ -1,3 +1,3 @@ -a array None -b double None -c map None \ No newline at end of file +a array +b double +c map diff --git a/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c index 6b1ce270d97e9..b584fd7c6fd36 100644 --- a/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c @@ -1,4 +1,4 @@ src srcpart test3a -test3b \ No newline at end of file +test3b diff --git a/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 b/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 index 92c0ed68f8a7b..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 +++ b/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ALTERTABLE_ADDCOLS TEST3b (TOK_TABCOLLIST (TOK_TABCOL X TOK_DOUBLE))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Alter Table Operator: - Alter Table - type: add columns - new columns: x double - old name: TEST3b - diff --git a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 index 594b29ca1410f..bd673a6c1f1d4 100644 --- a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 +++ b/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 @@ -1,4 +1,4 @@ -a array None -b double None -c map None -x double None \ No newline at end of file +a array +b double +c map +x double diff --git a/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be b/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be index 09bbc29377720..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be +++ b/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ALTERTABLE_RENAME TEST3b TEST3c) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Alter Table Operator: - Alter Table - type: rename - new name: TEST3c - old name: TEST3b - diff --git a/sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c b/sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab b/sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 b/sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 b/sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f b/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f deleted file mode 100644 index 25bf17fc5aaab..0000000000000 --- a/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f +++ /dev/null @@ -1 +0,0 @@ -18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe b/sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f b/sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 b/sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d b/sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 b/sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e b/sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 b/sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 b/sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 b/sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 b/sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 b/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 deleted file mode 100644 index 25bf17fc5aaab..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 +++ /dev/null @@ -1 +0,0 @@ -18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 b/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d b/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 b/sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 b/sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe b/sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 b/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 deleted file mode 100644 index 25bf17fc5aaab..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 +++ /dev/null @@ -1 +0,0 @@ -18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 b/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 b/sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 b/sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 b/sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 b/sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 b/sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d b/sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece b/sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d b/sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 b/sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 b/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 deleted file mode 100644 index 6731b0cb0baaa..0000000000000 --- a/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 +++ /dev/null @@ -1,20 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -0 val_1 -0 val_1 -1 val_2 -10 val_10 -10 val_11 -100 val_100 -100 val_100 -100 val_101 -100 val_101 -101 val_102 -102 val_103 -103 val_103 -103 val_103 -104 val_104 -104 val_104 -104 val_105 -104 val_105 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 b/sql/hive/src/test/resources/golden/input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 rename to sql/hive/src/test/resources/golden/input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e diff --git a/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 b/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 deleted file mode 100644 index 65a457b52b0a6..0000000000000 --- a/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 +++ /dev/null @@ -1,27 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_LOAD '/Users/marmbrus/workspace/hive/data/files/kv1.txt' (TOK_TAB (TOK_TABNAME INPUT4)) LOCAL) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - Stage-2 depends on stages: Stage-1 - -STAGE PLANS: - Stage: Stage-0 - Copy - source: file:/Users/marmbrus/workspace/hive/data/files/kv1.txt - destination: file:/tmp/hive-marmbrus/hive_2014-01-14_12-16-46_262_527870677085258278-1/-ext-10000 - - Stage: Stage-1 - Move Operator - tables: - replace: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.input4 - - Stage: Stage-2 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 b/sql/hive/src/test/resources/golden/input4-2-b663ec84da3f9d9b9594ea2da81b1442 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 rename to sql/hive/src/test/resources/golden/input4-2-b663ec84da3f9d9b9594ea2da81b1442 diff --git a/sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 b/sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 b/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 index 218c223b050b7..103b537db90c6 100644 --- a/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 +++ b/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 @@ -497,4 +497,4 @@ val_169 169 val_403 403 val_400 400 val_200 200 -val_97 97 \ No newline at end of file +val_97 97 diff --git a/sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb b/sql/hive/src/test/resources/golden/input40-1-a8adb8ae1d13607851431a1baf7578ba similarity index 100% rename from sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb rename to sql/hive/src/test/resources/golden/input40-1-a8adb8ae1d13607851431a1baf7578ba diff --git a/sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 b/sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 b/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 +++ b/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e b/sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 b/sql/hive/src/test/resources/golden/input40-4-fdeea6b676c670b17c8d91e24a97a127 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 rename to sql/hive/src/test/resources/golden/input40-4-fdeea6b676c670b17c8d91e24a97a127 diff --git a/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 b/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 index 6bc66cd8fe19b..e4b818f03539d 100644 --- a/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 +++ b/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 @@ -497,4 +497,4 @@ 97 val_97 2009-08-01 97 val_97 2009-08-01 98 val_98 2009-08-01 -98 val_98 2009-08-01 \ No newline at end of file +98 val_98 2009-08-01 diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 b/sql/hive/src/test/resources/golden/input40-6-6651f53efc5d03ed2d43b9d7aecc0002 similarity index 100% rename from sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 rename to sql/hive/src/test/resources/golden/input40-6-6651f53efc5d03ed2d43b9d7aecc0002 diff --git a/sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 b/sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 b/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 index ee3dddd8cff71..4467e7af00c0e 100644 --- a/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 +++ b/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 @@ -997,4 +997,4 @@ 97 val_98 2009-08-01 98 val_98 2009-08-01 98 val_98 2009-08-01 -99 val_100 2009-08-01 \ No newline at end of file +99 val_100 2009-08-01 diff --git a/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 +++ b/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee b/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee index 579784a58a66c..61191cde2953b 100644 --- a/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee +++ b/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee @@ -1,2 +1,2 @@ 0 -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a b/sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 b/sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f b/sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca b/sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 b/sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 b/sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab b/sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb b/sql/hive/src/test/resources/golden/input4_cb_delim-1-353d2238b781a117888a67bb7b2b2537 similarity index 100% rename from sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb rename to sql/hive/src/test/resources/golden/input4_cb_delim-1-353d2238b781a117888a67bb7b2b2537 diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 b/sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 b/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 index 218c223b050b7..103b537db90c6 100644 --- a/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 +++ b/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 @@ -497,4 +497,4 @@ val_169 169 val_403 403 val_400 400 val_200 200 -val_97 97 \ No newline at end of file +val_97 97 diff --git a/sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d b/sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af b/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af deleted file mode 100644 index 217a1915f8826..0000000000000 --- a/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af +++ /dev/null @@ -1,10 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 -5 val_5 -5 val_5 -5 val_5 -8 val_8 -9 val_9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea b/sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 b/sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad b/sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f8aa003a65bb4..0000000000000 --- a/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,11 +0,0 @@ -NULL NULL -[0,0,0] [{"myint":0,"mystring":"0","underscore_int":0}] -[1,2,3] [{"myint":1,"mystring":"1","underscore_int":1}] -[2,4,6] [{"myint":4,"mystring":"8","underscore_int":2}] -[3,6,9] [{"myint":9,"mystring":"27","underscore_int":3}] -[4,8,12] [{"myint":16,"mystring":"64","underscore_int":4}] -[5,10,15] [{"myint":25,"mystring":"125","underscore_int":5}] -[6,12,18] [{"myint":36,"mystring":"216","underscore_int":6}] -[7,14,21] [{"myint":49,"mystring":"343","underscore_int":7}] -[8,16,24] [{"myint":64,"mystring":"512","underscore_int":8}] -[9,18,27] [{"myint":81,"mystring":"729","underscore_int":9}] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f index b4dc488518f24..f3f63f08fcf70 100644 --- a/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ NULL val_265 NULL val_193 NULL NULL -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f index 0b8a8960a992b..65cada3d45b2b 100644 --- a/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f @@ -22,4 +22,4 @@ NULL 146 NULL 406 NULL NULL NULL NULL -NULL NULL \ No newline at end of file +NULL NULL diff --git a/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f index c87107a2f1168..416fbfb9e5228 100644 --- a/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f @@ -22,4 +22,4 @@ NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL -NULL NULL NULL \ No newline at end of file +NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e b/sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 b/sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be b/sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 b/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 deleted file mode 100644 index e782acd4d1e7f..0000000000000 --- a/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] ["0","0","0"] {"key_0":"value_0"} 1712634731 record_0 -[1,2,3] ["10","100","1000"] {"key_1":"value_1"} 465985200 record_1 -[2,4,6] ["20","200","2000"] {"key_2":"value_2"} -751827638 record_2 -[3,6,9] ["30","300","3000"] {"key_3":"value_3"} 477111222 record_3 -[4,8,12] ["40","400","4000"] {"key_4":"value_4"} -734328909 record_4 -[5,10,15] ["50","500","5000"] {"key_5":"value_5"} -1952710710 record_5 -[6,12,18] ["60","600","6000"] {"key_6":"value_6"} 1244525190 record_6 -[7,14,21] ["70","700","7000"] {"key_7":"value_7"} -1461153973 record_7 -[8,16,24] ["80","800","8000"] {"key_8":"value_8"} 1638581578 record_8 -[9,18,27] ["90","900","9000"] {"key_9":"value_9"} 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 b/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 deleted file mode 100644 index 6038b8aa32884..0000000000000 --- a/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 +++ /dev/null @@ -1,11 +0,0 @@ -0 0 NULL 1712634731 record_0 -1 10 NULL 465985200 record_1 -2 20 NULL -751827638 record_2 -3 30 NULL 477111222 record_3 -4 40 NULL -734328909 record_4 -5 50 NULL -1952710710 record_5 -6 60 NULL 1244525190 record_6 -7 70 NULL -1461153973 record_7 -8 80 NULL 1638581578 record_8 -9 90 NULL 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d b/sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc b/sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 b/sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index e782acd4d1e7f..0000000000000 --- a/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] ["0","0","0"] {"key_0":"value_0"} 1712634731 record_0 -[1,2,3] ["10","100","1000"] {"key_1":"value_1"} 465985200 record_1 -[2,4,6] ["20","200","2000"] {"key_2":"value_2"} -751827638 record_2 -[3,6,9] ["30","300","3000"] {"key_3":"value_3"} 477111222 record_3 -[4,8,12] ["40","400","4000"] {"key_4":"value_4"} -734328909 record_4 -[5,10,15] ["50","500","5000"] {"key_5":"value_5"} -1952710710 record_5 -[6,12,18] ["60","600","6000"] {"key_6":"value_6"} 1244525190 record_6 -[7,14,21] ["70","700","7000"] {"key_7":"value_7"} -1461153973 record_7 -[8,16,24] ["80","800","8000"] {"key_8":"value_8"} 1638581578 record_8 -[9,18,27] ["90","900","9000"] {"key_9":"value_9"} 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 b/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 deleted file mode 100644 index 6038b8aa32884..0000000000000 --- a/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 +++ /dev/null @@ -1,11 +0,0 @@ -0 0 NULL 1712634731 record_0 -1 10 NULL 465985200 record_1 -2 20 NULL -751827638 record_2 -3 30 NULL 477111222 record_3 -4 40 NULL -734328909 record_4 -5 50 NULL -1952710710 record_5 -6 60 NULL 1244525190 record_6 -7 70 NULL -1461153973 record_7 -8 80 NULL 1638581578 record_8 -9 90 NULL 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d b/sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 b/sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 b/sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c b/sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c b/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c deleted file mode 100644 index 6af528eab23a8..0000000000000 --- a/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c +++ /dev/null @@ -1,11 +0,0 @@ -{"key_0":"value_0"} -{"key_1":"value_1"} -{"key_2":"value_2"} -{"key_3":"value_3"} -{"key_4":"value_4"} -{"key_5":"value_5"} -{"key_6":"value_6"} -{"key_7":"value_7"} -{"key_8":"value_8"} -{"key_9":"value_9"} -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc b/sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 b/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 deleted file mode 100644 index e782acd4d1e7f..0000000000000 --- a/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] ["0","0","0"] {"key_0":"value_0"} 1712634731 record_0 -[1,2,3] ["10","100","1000"] {"key_1":"value_1"} 465985200 record_1 -[2,4,6] ["20","200","2000"] {"key_2":"value_2"} -751827638 record_2 -[3,6,9] ["30","300","3000"] {"key_3":"value_3"} 477111222 record_3 -[4,8,12] ["40","400","4000"] {"key_4":"value_4"} -734328909 record_4 -[5,10,15] ["50","500","5000"] {"key_5":"value_5"} -1952710710 record_5 -[6,12,18] ["60","600","6000"] {"key_6":"value_6"} 1244525190 record_6 -[7,14,21] ["70","700","7000"] {"key_7":"value_7"} -1461153973 record_7 -[8,16,24] ["80","800","8000"] {"key_8":"value_8"} 1638581578 record_8 -[9,18,27] ["90","900","9000"] {"key_9":"value_9"} 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db b/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db deleted file mode 100644 index 6038b8aa32884..0000000000000 --- a/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db +++ /dev/null @@ -1,11 +0,0 @@ -0 0 NULL 1712634731 record_0 -1 10 NULL 465985200 record_1 -2 20 NULL -751827638 record_2 -3 30 NULL 477111222 record_3 -4 40 NULL -734328909 record_4 -5 50 NULL -1952710710 record_5 -6 60 NULL 1244525190 record_6 -7 70 NULL -1461153973 record_7 -8 80 NULL 1638581578 record_8 -9 90 NULL 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 b/sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 b/sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c b/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c deleted file mode 100644 index 1bb008b44d6ee..0000000000000 --- a/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] -[1,2,3] -[2,4,6] -[3,6,9] -[4,8,12] -[5,10,15] -[6,12,18] -[7,14,21] -[8,16,24] -[9,18,27] -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 b/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 index 17f24d2991d14..03887aed65852 100644 --- a/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 +++ b/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 @@ -17,4 +17,4 @@ 369 val_369 66 val_66 128 val_128 -213 val_213 \ No newline at end of file +213 val_213 diff --git a/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb b/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb index 499e6b8ab6b7e..d7a8f25b41301 100644 --- a/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb +++ b/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb @@ -997,4 +997,4 @@ 403 val_403 2008-04-08 12 400 val_400 2008-04-08 12 200 val_200 2008-04-08 12 -97 val_97 2008-04-08 12 \ No newline at end of file +97 val_97 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f index 3e5ae10e4670a..7ae7ecbe28de6 100644 --- a/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 28 val_28 12 2008-04-08 37 val_37 12 2008-04-08 90 val_90 12 2008-04-08 -97 val_97 12 2008-04-08 \ No newline at end of file +97 val_97 12 2008-04-08 diff --git a/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 b/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 index 82116102c1f54..89c49ce857f5a 100644 --- a/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 +++ b/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 @@ -1,12 +1,12 @@ -a string None -b string None -ds string None -ts string None +a string +b string +ds string +ts string # Partition Information # col_name data_type comment -ds string None -ts string None +ds string +ts string -Detailed Partition Information Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1388798899, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6540137288252557391/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388798899}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1413882241, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882241, COLUMN_STATS_ACCURATE=true, totalSize=4, numRows=1, rawDataSize=3}) diff --git a/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 b/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 index c8d0d55930069..6de1c02821c77 100644 --- a/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 +++ b/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 @@ -1 +1 @@ -1 2 2008 04 08 10:11:12=455 \ No newline at end of file +1 2 2008 04 08 10:11:12=455 diff --git a/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 b/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 index 869eb58e70d82..f5c60fae1925e 100644 --- a/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 +++ b/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 @@ -1,12 +1,12 @@ -a string None -b string None -ds string None -ts string None +a string +b string +ds string +ts string # Partition Information # col_name data_type comment -ds string None -ts string None +ds string +ts string -Detailed Partition Information Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1388798920, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6540137288252557391/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1388798920, numRows=1, totalSize=4, rawDataSize=3}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1413882252, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882252, COLUMN_STATS_ACCURATE=true, totalSize=4, numRows=1, rawDataSize=3}) diff --git a/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 b/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 index c8d0d55930069..6de1c02821c77 100644 --- a/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 +++ b/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 @@ -1 +1 @@ -1 2 2008 04 08 10:11:12=455 \ No newline at end of file +1 2 2008 04 08 10:11:12=455 diff --git a/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 b/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 index fd945cc15d9ca..883de3e945c46 100644 --- a/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 +++ b/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 @@ -81,4 +81,4 @@ 97 val_97 12 2008-04-08 97 val_97 12 2008-04-08 98 val_98 12 2008-04-08 -98 val_98 12 2008-04-08 \ No newline at end of file +98 val_98 12 2008-04-08 diff --git a/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 b/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 index f23877361f33b..b23aa27263654 100644 --- a/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 +++ b/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 @@ -81,4 +81,4 @@ 97 val_97 12 2008-04-09 97 val_97 12 2008-04-09 98 val_98 12 2008-04-09 -98 val_98 12 2008-04-09 \ No newline at end of file +98 val_98 12 2008-04-09 diff --git a/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 b/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 index f4026a591a958..31b575a403f81 100644 --- a/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 +++ b/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 @@ -497,4 +497,4 @@ 403 val_403 2008-04-08 11 400 val_400 2008-04-08 11 200 val_200 2008-04-08 11 -97 val_97 2008-04-08 11 \ No newline at end of file +97 val_97 2008-04-08 11 diff --git a/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 b/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 index 853c3bc8df7f0..dfabe7827837c 100644 --- a/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 +++ b/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 @@ -165,4 +165,4 @@ 98 val_98 2008-04-08 11 98 val_98 2008-04-08 11 98 val_98 2008-04-08 12 -98 val_98 2008-04-08 12 \ No newline at end of file +98 val_98 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 b/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 index 540ca86636f3c..ed8993a9cbd0c 100644 --- a/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 +++ b/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 @@ -333,4 +333,4 @@ 98 val_98 2008-04-08 12 98 val_98 2008-04-08 12 98 val_98 2008-04-08 12 -98 val_98 2008-04-08 12 \ No newline at end of file +98 val_98 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 b/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 index a1728c82f0b35..5ee171a64f7ab 100644 --- a/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 +++ b/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 @@ -7,4 +7,4 @@ 255 val_255 2008-04-08 11 278 val_278 2008-04-08 11 98 val_98 2008-04-08 11 -484 val_484 2008-04-08 11 \ No newline at end of file +484 val_484 2008-04-08 11 diff --git a/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 b/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 index 438355d7b06f5..b7704cd0a2f0f 100644 --- a/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 +++ b/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 @@ -997,4 +997,4 @@ 498 val_498 2008-04-08 11 498 val_498 2008-04-08 12 498 val_498 2008-04-08 12 -498 val_498 2008-04-08 12 \ No newline at end of file +498 val_498 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e b/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 b/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 b/sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f b/sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 b/sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index c1a6abba038e7..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,11 +0,0 @@ -0 0 NULL -2 1 NULL -4 8 value_2 -6 27 NULL -8 64 NULL -10 125 NULL -12 216 NULL -14 343 NULL -16 512 NULL -18 729 NULL -NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 b/sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 b/sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd b/sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 51645b2a07a39..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac b/sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 b/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 deleted file mode 100644 index 373a573714f4e..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 +++ /dev/null @@ -1,11 +0,0 @@ -NULL [0] -NULL [1] -NULL [4] -NULL [9] -NULL [16] -NULL [25] -NULL [36] -NULL [49] -NULL [64] -value_9 [81] -NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 b/sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 b/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 deleted file mode 100644 index 7490d2d44d71a..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 +++ /dev/null @@ -1,10 +0,0 @@ -NULL [0] -NULL [1] -NULL [4] -NULL [9] -NULL [16] -NULL [25] -NULL [36] -NULL [49] -NULL [64] -value_9 [81] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 b/sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 b/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 deleted file mode 100644 index 7490d2d44d71a..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 +++ /dev/null @@ -1,10 +0,0 @@ -NULL [0] -NULL [1] -NULL [4] -NULL [9] -NULL [16] -NULL [25] -NULL [36] -NULL [49] -NULL [64] -value_9 [81] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 index 679d54cb5cb5e..3577c8a431869 100644 --- a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 +++ b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 @@ -1,10 +1,10 @@ -key int None -value string None -ds string None -country string None +key int +value string +ds string +country string # Partition Information # col_name data_type comment -ds string None -country string None +ds string +country string diff --git a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c index ded361eb294f0..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c +++ b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c @@ -1,2 +1,2 @@ -key int None -value string None +key int +value string diff --git a/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 b/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 index d5a489a9a5ed5..454e27ff0c28d 100644 --- a/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 +++ b/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 @@ -1,15 +1,15 @@ -viewtime string None -userid int None -page_url string None -referrer_url string None -friends array None -properties map None +viewtime string +userid int +page_url string +referrer_url string +friends array +properties map ip string IP Address of the User -ds string None -country string None +ds string +country string # Partition Information # col_name data_type comment -ds string None -country string None \ No newline at end of file +ds string +country string diff --git a/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 b/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 index e0bedb0512cfc..0ea7cee2a9cf9 100644 --- a/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 +++ b/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 @@ -1,17 +1,17 @@ -viewtime string None -userid int None -page_url string None -referrer_url string None -friends array None -properties map None +viewtime string +userid int +page_url string +referrer_url string +friends array +properties map ip string IP Address of the User -ds string None -country string None +ds string +country string # Partition Information # col_name data_type comment -ds string None -country string None +ds string +country string -Detailed Table Information Table(tableName:inputddl4, dbName:default, owner:marmbrus, createTime:1389731336, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:viewtime, type:string, comment:null), FieldSchema(name:userid, type:int, comment:null), FieldSchema(name:page_url, type:string, comment:null), FieldSchema(name:referrer_url, type:string, comment:null), FieldSchema(name:friends, type:array, comment:null), FieldSchema(name:properties, type:map, comment:null), FieldSchema(name:ip, type:string, comment:IP Address of the User), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[userid], sortCols:[Order(col:viewtime, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1389731336, comment=This is the page view table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:inputddl4, dbName:default, owner:marmbrus, createTime:1413882343, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:viewtime, type:string, comment:null), FieldSchema(name:userid, type:int, comment:null), FieldSchema(name:page_url, type:string, comment:null), FieldSchema(name:referrer_url, type:string, comment:null), FieldSchema(name:friends, type:array, comment:null), FieldSchema(name:properties, type:map, comment:null), FieldSchema(name:ip, type:string, comment:IP Address of the User), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[userid], sortCols:[Order(col:viewtime, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882343, comment=This is the page view table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 b/sql/hive/src/test/resources/golden/inputddl6-1-572c1abb70f09726d1ba77bdc884597b similarity index 100% rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 rename to sql/hive/src/test/resources/golden/inputddl6-1-572c1abb70f09726d1ba77bdc884597b diff --git a/sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf b/sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 b/sql/hive/src/test/resources/golden/inputddl6-2-3a4def4b370f75c5fcc1174626490363 similarity index 100% rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 rename to sql/hive/src/test/resources/golden/inputddl6-2-3a4def4b370f75c5fcc1174626490363 diff --git a/sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c b/sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c b/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c index f1427896e8330..a8a418d9736d9 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c +++ b/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:inputddl6, dbName:default, owner:marmbrus, createTime:1389731342, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl6, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=2, numFiles=2, transient_lastDdlTime=1389731342, numRows=0, totalSize=11624, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:inputddl6, dbName:default, owner:marmbrus, createTime:1413882344, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl6, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882344}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 b/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 index 822897217e867..05507162a9244 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 +++ b/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2008-04-08], dbName:default, tableName:inputddl6, createTime:1389731342, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl6/ds=2008-04-08, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731342, numRows=0, totalSize=5812, rawDataSize=0}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008-04-08], dbName:default, tableName:inputddl6, createTime:1413882344, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl6/ds=2008-04-08, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882344, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0}) diff --git a/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 b/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 index 94bcaaee2408c..6e2459ea22ee1 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 +++ b/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 @@ -1,2 +1,2 @@ ds=2008-04-08 -ds=2008-04-09 \ No newline at end of file +ds=2008-04-09 diff --git a/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 b/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 index b12a9f82cd90a..017a142ab30b7 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 +++ b/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 @@ -1 +1 @@ -ds=2008-04-09 \ No newline at end of file +ds=2008-04-09 diff --git a/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 b/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 index a6c282ab6f573..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 +++ b/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_DESCTABLE (TOK_TABTYPE INPUTDDL6 (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09'))) EXTENDED) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Describe Table Operator: - Describe Table - partition: - ds 2008-04-09 - table: INPUTDDL6 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 b/sql/hive/src/test/resources/golden/inputddl7-1-7195712efb4910294f63303ebce24453 similarity index 100% rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 rename to sql/hive/src/test/resources/golden/inputddl7-1-7195712efb4910294f63303ebce24453 diff --git a/sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 b/sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d b/sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c b/sql/hive/src/test/resources/golden/inputddl7-10-7c9248b56948716913d332bd712d69bd similarity index 100% rename from sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c rename to sql/hive/src/test/resources/golden/inputddl7-10-7c9248b56948716913d332bd712d69bd diff --git a/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 b/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 +++ b/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 b/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 index ef633a4aa09e0..edfcdbb1211bb 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 +++ b/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 @@ -1,3 +1,3 @@ -name string None +name string -Detailed Table Information Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1389731349, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, transient_lastDdlTime=1389731349, numRows=0, totalSize=5812, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1413882345, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413882345, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a b/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a index 86d5d8a125fbe..86c9b459e36c5 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a +++ b/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a @@ -1,3 +1,3 @@ -name string None +name string -Detailed Table Information Table(tableName:t2, dbName:default, owner:marmbrus, createTime:1389731362, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t2, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, transient_lastDdlTime=1389731362, numRows=0, totalSize=10508, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:t2, dbName:default, owner:marmbrus, createTime:1413882355, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t2, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413882355, COLUMN_STATS_ACCURATE=true, totalSize=10508, numRows=0, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 b/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 index b14d78536150c..04956c2c1c447 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 +++ b/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 @@ -1,9 +1,9 @@ -name string None -ds string None +name string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2008-04-09], dbName:default, tableName:t3, createTime:1389731375, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t3/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731375, numRows=0, totalSize=5812, rawDataSize=0}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008-04-09], dbName:default, tableName:t3, createTime:1413882365, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t3/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882365, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0}) diff --git a/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe b/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe index e476a7fa3959b..76ecadd3851a6 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe +++ b/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe @@ -1,9 +1,9 @@ -name string None -ds string None +name string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2008-04-09], dbName:default, tableName:t4, createTime:1389731388, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t4/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731388, numRows=0, totalSize=10508, rawDataSize=0}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008-04-09], dbName:default, tableName:t4, createTime:1413882375, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t4/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882375, COLUMN_STATS_ACCURATE=true, totalSize=10508, numRows=0, rawDataSize=0}) diff --git a/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 b/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 +++ b/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 b/sql/hive/src/test/resources/golden/inputddl7-4-68715ba2c11220be62394c86453e6d54 similarity index 100% rename from sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 rename to sql/hive/src/test/resources/golden/inputddl7-4-68715ba2c11220be62394c86453e6d54 diff --git a/sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 b/sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 b/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 +++ b/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 b/sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 b/sql/hive/src/test/resources/golden/inputddl7-7-59dd2d2556769e19bdc0a444f40f8a71 similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 rename to sql/hive/src/test/resources/golden/inputddl7-7-59dd2d2556769e19bdc0a444f40f8a71 diff --git a/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 b/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 +++ b/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 b/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 index 66e6efceed3dc..5166f3678f405 100644 --- a/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 +++ b/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 @@ -4,13 +4,13 @@ lint array from deserializer lstring array from deserializer lintstring array> from deserializer mstringstring map from deserializer -ds string None -country string None +ds string +country string # Partition Information # col_name data_type comment -ds string None -country string None +ds string +country string -Detailed Table Information Table(tableName:inputddl8, dbName:default, owner:marmbrus, createTime:1389731407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl8, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.Complex, serialization.format=com.facebook.thrift.protocol.TBinaryProtocol}), bucketCols:[aint], sortCols:[Order(col:lint, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1389731407, comment=This is a thrift based table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:inputddl8, dbName:default, owner:marmbrus, createTime:1413882387, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl8, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.Complex, serialization.format=com.facebook.thrift.protocol.TBinaryProtocol}), bucketCols:[aint], sortCols:[Order(col:lint, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882387, comment=This is a thrift based table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-86653b3af59df59f225ee00ff5fc119f similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 rename to sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-86653b3af59df59f225ee00ff5fc119f diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-e81d45a5bec5642ec4b762f1c1a482af similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 rename to sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-e81d45a5bec5642ec4b762f1c1a482af diff --git a/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d +++ b/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef +++ b/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef index e37d32abba426..83b33d238dab9 100644 --- a/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef +++ b/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef @@ -1 +1 @@ -1000 \ No newline at end of file +1000 diff --git a/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef index 37021f4a27201..3d86ec6498f3f 100644 --- a/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef +++ b/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef @@ -1 +1 @@ -1500 \ No newline at end of file +1500 diff --git a/sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 b/sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b b/sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 b/sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb deleted file mode 100644 index 5e96d815b6b78..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb +++ /dev/null @@ -1 +0,0 @@ --826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 b/sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 b/sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 b/sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb deleted file mode 100644 index eb9dc5833c2f9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb +++ /dev/null @@ -1 +0,0 @@ -10226524244 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 b/sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 b/sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb deleted file mode 100644 index 28ced898ab537..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb +++ /dev/null @@ -1 +0,0 @@ -20453048488 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d b/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d deleted file mode 100644 index ae4ee13c08e76..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d +++ /dev/null @@ -1 +0,0 @@ -200 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b b/sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 b/sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 b/sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 b/sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa b/sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 deleted file mode 100644 index 84d7f3929d86d..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 +++ /dev/null @@ -1 +0,0 @@ --27100860056 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 b/sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 b/sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 b/sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 b/sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 b/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 deleted file mode 100644 index ae4ee13c08e76..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 +++ /dev/null @@ -1 +0,0 @@ -200 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 deleted file mode 100644 index 3395f3bcc7b51..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 +++ /dev/null @@ -1 +0,0 @@ --24159954504 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 b/sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 b/sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 deleted file mode 100644 index ee0a47c9f6e00..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 +++ /dev/null @@ -1 +0,0 @@ --36239931656 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 b/sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 b/sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 b/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 deleted file mode 100644 index 5e96d815b6b78..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 +++ /dev/null @@ -1 +0,0 @@ --826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 b/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 deleted file mode 100644 index 2ed5a7da11dcd..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 +++ /dev/null @@ -1 +0,0 @@ -9399898328 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 b/sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 b/sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 b/sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d b/sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd b/sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 b/sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 b/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 deleted file mode 100644 index 00ffdd24b0cff..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 +++ /dev/null @@ -1 +0,0 @@ -7813690682 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 b/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 deleted file mode 100644 index eb9dc5833c2f9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 +++ /dev/null @@ -1 +0,0 @@ -10226524244 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 b/sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 b/sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 b/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 b/sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 b/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 deleted file mode 100644 index e2954bd63682e..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 +++ /dev/null @@ -1 +0,0 @@ --1653251832 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa b/sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 b/sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b b/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b deleted file mode 100644 index e2954bd63682e..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b +++ /dev/null @@ -1 +0,0 @@ --1653251832 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 b/sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 b/sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 b/sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b b/sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a b/sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 b/sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 b/sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 b/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 deleted file mode 100644 index 5e96d815b6b78..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 +++ /dev/null @@ -1 +0,0 @@ --826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 b/sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 b/sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 b/sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c b/sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 b/sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d b/sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 b/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 deleted file mode 100644 index 3b6ef434b37e7..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 +++ /dev/null @@ -1 +0,0 @@ --18626052920 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 b/sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 b/sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 b/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 deleted file mode 100644 index 0744f3dae0e0a..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 +++ /dev/null @@ -1 +0,0 @@ --37252105840 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 b/sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c b/sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 b/sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 b/sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 b/sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 b/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 deleted file mode 100644 index c0066b75af40e..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 +++ /dev/null @@ -1 +0,0 @@ -481928560 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f b/sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 b/sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 b/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 deleted file mode 100644 index 0b4a44e064f85..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 +++ /dev/null @@ -1 +0,0 @@ -963857120 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 b/sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba b/sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 b/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 deleted file mode 100644 index d541b5d207233..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 +++ /dev/null @@ -1 +0,0 @@ --35226404960 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 b/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 deleted file mode 100644 index c21f4017362c1..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 +++ /dev/null @@ -1,2 +0,0 @@ -ds=1 -ds=2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 b/sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 b/sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f b/sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 b/sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 b/sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 b/sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 b/sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab b/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab deleted file mode 100644 index d541b5d207233..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab +++ /dev/null @@ -1 +0,0 @@ --35226404960 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 b/sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a b/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b b/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 b/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 b/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 index 5707ed08e7e54..13e88f30fc08c 100644 --- a/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 +++ b/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 @@ -97,4 +97,4 @@ 9 val_9 5 val_5 9 val_9 5 val_5 9 val_9 8 val_8 -9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 b/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 +++ b/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d b/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d +++ b/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 b/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 index c888cfd6a0479..e9016953d4e48 100644 --- a/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 +++ b/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 @@ -145,4 +145,4 @@ 90 val_90 90 val_90 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f b/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f index 1780076ae513f..598608f124d38 100644 --- a/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f +++ b/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f @@ -229,4 +229,4 @@ 37 val_37 37 val_37 37 val_37 -37 val_37 \ No newline at end of file +37 val_37 diff --git a/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 b/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 index a82a9a22c6b87..f9890a4c2d194 100644 --- a/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 +++ b/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 @@ -161,4 +161,4 @@ 90 val_90 90 val_90 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e +++ b/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 +++ b/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 +++ b/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 b/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 +++ b/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 index 941e6bb7c887e..bd6afa8dada84 100644 --- a/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 +++ b/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 @@ -1749,4 +1749,4 @@ 403 val_403 400 val_400 200 val_200 -200 val_200 \ No newline at end of file +200 val_200 diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f +++ b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 +++ b/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 index 941e6bb7c887e..bd6afa8dada84 100644 --- a/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 +++ b/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 @@ -1749,4 +1749,4 @@ 403 val_403 400 val_400 200 val_200 -200 val_200 \ No newline at end of file +200 val_200 diff --git a/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 b/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 index c2e3ea8b0c8e2..b212e93a0a8c2 100644 --- a/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 +++ b/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 @@ -1025,4 +1025,4 @@ 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 -498 val_498 498 val_498 \ No newline at end of file +498 val_498 498 val_498 diff --git a/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f index 66fafbdf72b63..3df4716f0b05f 100644 --- a/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 200 val_200 200 val_200 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 b/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 index 10628f6c64bd4..947ed199b0d4d 100644 --- a/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 +++ b/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 @@ -307,4 +307,4 @@ NULL NULL NULL 7 495 1 NULL NULL 496 1 NULL NULL 497 1 NULL NULL -498 3 NULL NULL \ No newline at end of file +498 3 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 b/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 index 2945c31ed4f00..12a176c7bc7cb 100644 --- a/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 +++ b/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 @@ -469,4 +469,4 @@ 200 val_400 200 val_400 97 val_194 -97 val_194 \ No newline at end of file +97 val_194 diff --git a/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 b/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 index bd6e20b8de1e4..7568f8cde28f5 100644 --- a/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 +++ b/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 @@ -545,4 +545,4 @@ NULL NULL NULL NULL 498 val_498 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 8 val_8 8 val_8 8 val_8 -9 val_9 9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 b/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 index bd6e20b8de1e4..7568f8cde28f5 100644 --- a/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 +++ b/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 @@ -545,4 +545,4 @@ NULL NULL NULL NULL 498 val_498 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 8 val_8 8 val_8 8 val_8 -9 val_9 9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f b/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f index 9672e21fa0323..80c230cf4a09d 100644 --- a/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f +++ b/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f @@ -497,4 +497,4 @@ NULL NULL NULL NULL 496 val_496 NULL NULL NULL NULL 497 val_497 NULL NULL NULL NULL 498 val_498 NULL NULL NULL NULL 498 val_498 -NULL NULL NULL NULL 498 val_498 \ No newline at end of file +NULL NULL NULL NULL 498 val_498 diff --git a/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 b/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 index 5707ed08e7e54..13e88f30fc08c 100644 --- a/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 +++ b/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 @@ -97,4 +97,4 @@ 9 val_9 5 val_5 9 val_9 5 val_5 9 val_9 8 val_8 -9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 b/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 +++ b/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 +++ b/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa index 183353e5c705b..d3d377e8ae74d 100644 --- a/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -34,4 +34,4 @@ 406 val_406 val_406 406 val_406 val_406 406 val_406 val_406 -406 val_406 val_406 \ No newline at end of file +406 val_406 val_406 diff --git a/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa index 4c88927e56579..120056ea10c60 100644 --- a/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -104,4 +104,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 b/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 index 2b75023ea4a9f..2adcbc0f14b62 100644 --- a/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 +++ b/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 @@ -38,4 +38,4 @@ NULL val_484 val_484 406 val_406 val_406 406 val_406 val_406 406 val_406 val_406 -406 val_406 val_406 \ No newline at end of file +406 val_406 val_406 diff --git a/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa index da3c427cab73f..c998494e1c524 100644 --- a/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -104,4 +104,4 @@ 98 val_98 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa index f553ce0ca41f3..c8445b6e2c784 100644 --- a/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -12,4 +12,4 @@ 401 1 5 406 1 4 66 1 1 -98 1 2 \ No newline at end of file +98 1 2 diff --git a/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f index 8886dc7e8f229..9c33812fa7ea4 100644 --- a/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f @@ -2651,4 +2651,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa index 71094ee7360db..16b313fc58f23 100644 --- a/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -12,4 +12,4 @@ 311 3 369 3 401 5 -406 4 \ No newline at end of file +406 4 diff --git a/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa index 5d28208ab255c..7c33b34887d6b 100644 --- a/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -12,4 +12,4 @@ 401 1 406 1 66 1 -98 1 \ No newline at end of file +98 1 diff --git a/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a +++ b/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 index b1251794645ce..13c35f8a6c6ba 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 @@ -94,4 +94,4 @@ NULL val_484 val_484 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 b/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a +++ b/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa index e18d1ff802c93..6fbe456119b70 100644 --- a/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -31,4 +31,4 @@ 406 val_406 val_406 406 val_406 val_406 406 val_406 val_406 -406 val_406 val_406 \ No newline at end of file +406 val_406 val_406 diff --git a/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa index efdd80c9f8b89..2f62508e3342a 100644 --- a/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -10,4 +10,4 @@ 311 val_311 3 369 3 401 val_401 5 -406 val_406 4 \ No newline at end of file +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 +++ b/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa index a250f202c4df0..4cb7ec20b1a33 100644 --- a/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa @@ -306,4 +306,4 @@ 495 1 1 496 1 1 497 1 1 -498 3 3 \ No newline at end of file +498 3 3 diff --git a/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 +++ b/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa index 183353e5c705b..d3d377e8ae74d 100644 --- a/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -34,4 +34,4 @@ 406 val_406 val_406 406 val_406 val_406 406 val_406 val_406 -406 val_406 val_406 \ No newline at end of file +406 val_406 val_406 diff --git a/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 b/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 index 5e020e3aff653..3dcf0e17586a9 100644 --- a/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 +++ b/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 @@ -1,2 +1,2 @@ 100 101 102 103 104 105 106 107 108 109 110 111 -100 101 102 103 104 105 106 107 108 109 110 111 \ No newline at end of file +100 101 102 103 104 105 106 107 108 109 110 111 diff --git a/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef b/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef index fce97c9211916..edbff1207ded9 100644 --- a/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef +++ b/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef @@ -1 +1 @@ -val_111 105 2 \ No newline at end of file +val_111 105 2 diff --git a/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa index 5b45955ceb274..56d4dbe5b4d25 100644 --- a/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -563,4 +563,4 @@ 98 val_98 98 val_98 98 val_98 98 val_98 98 val_98 98 val_98 -98 val_98 98 val_98 \ No newline at end of file +98 val_98 98 val_98 diff --git a/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f index e45f3234d5a88..dc3a273d7a367 100644 --- a/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f @@ -8,4 +8,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac b/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac +++ b/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa b/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa index 8543fe173f87f..50237fbde9e34 100644 --- a/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa +++ b/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa @@ -563,4 +563,4 @@ 400 val_400 NULL NULL 200 val_200 NULL NULL 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda b/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda index 67d892c80f493..1f3d8a7a1fc08 100644 --- a/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda +++ b/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda @@ -1 +1 @@ -1028 \ No newline at end of file +1028 diff --git a/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 b/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 index 8543fe173f87f..50237fbde9e34 100644 --- a/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 +++ b/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 @@ -563,4 +563,4 @@ 400 val_400 NULL NULL 200 val_200 NULL NULL 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 b/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 +++ b/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 b/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 index bd6e20b8de1e4..7568f8cde28f5 100644 --- a/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 +++ b/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 @@ -545,4 +545,4 @@ NULL NULL NULL NULL 498 val_498 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 8 val_8 8 val_8 8 val_8 -9 val_9 9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 b/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 index bd6e20b8de1e4..7568f8cde28f5 100644 --- a/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 +++ b/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 @@ -545,4 +545,4 @@ NULL NULL NULL NULL 498 val_498 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 8 val_8 8 val_8 8 val_8 -9 val_9 9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 b/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 +++ b/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 b/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 index 7265626e5dbde..c20eb16d47f91 100644 --- a/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 +++ b/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 @@ -1,3 +1,3 @@ 0 val_0 NULL NULL 0 val_0 NULL NULL -0 val_0 NULL NULL \ No newline at end of file +0 val_0 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 b/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 index 7265626e5dbde..c20eb16d47f91 100644 --- a/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 +++ b/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 @@ -1,3 +1,3 @@ 0 val_0 NULL NULL 0 val_0 NULL NULL -0 val_0 NULL NULL \ No newline at end of file +0 val_0 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f index 06b2b4d7e6d47..b52cff5c472e4 100644 --- a/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f @@ -6,4 +6,4 @@ NULL NULL 24 val_24 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f index 36ca4a0805f4b..fb58885263569 100644 --- a/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f @@ -11,4 +11,4 @@ 19 val_19 19 val_19 NULL NULL 20 val_20 NULL NULL 24 val_24 -NULL NULL 24 val_24 \ No newline at end of file +NULL NULL 24 val_24 diff --git a/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f index eb0a6246c1988..0e75c1b63abb5 100644 --- a/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f @@ -11,4 +11,4 @@ 19 val_19 19 val_19 NULL NULL NULL NULL 20 val_20 NULL NULL NULL NULL 24 val_24 NULL NULL -NULL NULL 24 val_24 NULL NULL \ No newline at end of file +NULL NULL 24 val_24 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f index f2e6e07d8fa1f..c3f5359beb06f 100644 --- a/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f @@ -2,4 +2,4 @@ 12 val_12 NULL NULL 15 val_15 NULL NULL 11 val_11 NULL NULL -12 val_12 NULL NULL \ No newline at end of file +12 val_12 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a b/sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 b/sql/hive/src/test/resources/golden/join_1to1-1-789b2636cfb6a08965e0bd190e419762 similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 rename to sql/hive/src/test/resources/golden/join_1to1-1-789b2636cfb6a08965e0bd190e419762 diff --git a/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e b/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e +++ b/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 +++ b/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 b/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 +++ b/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e b/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e +++ b/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 b/sql/hive/src/test/resources/golden/join_1to1-3-6228e662e573a00ed04550d049d97a3b similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 rename to sql/hive/src/test/resources/golden/join_1to1-3-6228e662e573a00ed04550d049d97a3b diff --git a/sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 b/sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 +++ b/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 b/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 +++ b/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 b/sql/hive/src/test/resources/golden/join_array-2-16840a0266cad03a1a0b134d105b854f similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 rename to sql/hive/src/test/resources/golden/join_array-2-16840a0266cad03a1a0b134d105b854f diff --git a/sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de b/sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 b/sql/hive/src/test/resources/golden/join_array-3-a6ca6b64324596831033fdfe5b63a942 similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 rename to sql/hive/src/test/resources/golden/join_array-3-a6ca6b64324596831033fdfe5b63a942 diff --git a/sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 b/sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 b/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 index ce9688a065b34..84660b3eb4327 100644 --- a/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 +++ b/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 @@ -1 +1 @@ -10320092026892491 3312 \ No newline at end of file +10320092026892491 3312 diff --git a/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a b/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a index f937af554adef..1d1c48ae5a48f 100644 --- a/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a +++ b/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a @@ -1,2 +1,2 @@ 10320092002467760 [0,23,37,48,53,55,55,56,60,66,72,76,77,78,80,81,87,88,90,90,91,90,92,97,100,103,104,107,108,108,109,110,113,113,113,113,113,113,114,116,116,116,117,116,117,117,117,115,115,117,117,117,121,120,131,131,131,125,125,124,124,128,128,131,131,132,133,134,134,134,134,26,26,null,null,null,null,116] -10320092026892491 [0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139] \ No newline at end of file +10320092026892491 [0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139] diff --git a/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c b/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c index f937af554adef..1d1c48ae5a48f 100644 --- a/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c +++ b/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c @@ -1,2 +1,2 @@ 10320092002467760 [0,23,37,48,53,55,55,56,60,66,72,76,77,78,80,81,87,88,90,90,91,90,92,97,100,103,104,107,108,108,109,110,113,113,113,113,113,113,114,116,116,116,117,116,117,117,117,115,115,117,117,117,121,120,131,131,131,125,125,124,124,128,128,131,131,132,133,134,134,134,134,26,26,null,null,null,null,116] -10320092026892491 [0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139] \ No newline at end of file +10320092026892491 [0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139] diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 b/sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 b/sql/hive/src/test/resources/golden/join_casesensitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 rename to sql/hive/src/test/resources/golden/join_casesensitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc b/sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 b/sql/hive/src/test/resources/golden/join_casesensitive-3-93300f4a9242fa2804a5b368538d83f similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 rename to sql/hive/src/test/resources/golden/join_casesensitive-3-93300f4a9242fa2804a5b368538d83f diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd b/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd index 810f01f682c37..f7d190a11d65c 100644 --- a/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd +++ b/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd @@ -82,4 +82,4 @@ NULL 10050 66 50 10050 88 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a b/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a +++ b/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 b/sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d b/sql/hive/src/test/resources/golden/join_filters-1-bee6095f42de6a16708c2f9addc1b9bd similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d rename to sql/hive/src/test/resources/golden/join_filters-1-bee6095f42de6a16708c2f9addc1b9bd diff --git a/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 b/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 +++ b/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe b/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe +++ b/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b b/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b +++ b/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 b/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 +++ b/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b b/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b +++ b/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 b/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 +++ b/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb b/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb +++ b/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 b/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 +++ b/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 b/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 +++ b/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e b/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e +++ b/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 b/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 +++ b/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 b/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 +++ b/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba b/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba +++ b/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 b/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 +++ b/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 b/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 +++ b/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba b/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba +++ b/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 b/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 +++ b/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 b/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 +++ b/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 b/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 +++ b/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 b/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 +++ b/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e b/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e +++ b/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a b/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a +++ b/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e b/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e +++ b/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 b/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 +++ b/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada b/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada +++ b/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 b/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 +++ b/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 b/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 +++ b/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf b/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf +++ b/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 b/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 +++ b/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 b/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 +++ b/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 b/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 +++ b/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 b/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 +++ b/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f b/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f +++ b/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 b/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 +++ b/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf b/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf +++ b/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e b/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e +++ b/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e b/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e +++ b/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f b/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f index ea646edf6d143..04ecda59b21bd 100644 --- a/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f +++ b/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL NULL NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 b/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 +++ b/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 b/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 +++ b/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca b/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca index ea646edf6d143..04ecda59b21bd 100644 --- a/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca +++ b/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL NULL NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 b/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 +++ b/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 b/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 +++ b/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 b/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 +++ b/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 b/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 +++ b/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf b/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf +++ b/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f b/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f +++ b/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab b/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab +++ b/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 b/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 +++ b/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 b/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 +++ b/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe b/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe +++ b/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b b/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b +++ b/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 b/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 +++ b/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b b/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b +++ b/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 b/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 +++ b/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 b/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 +++ b/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb b/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb +++ b/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 b/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 +++ b/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 b/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 +++ b/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e b/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e +++ b/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f b/sql/hive/src/test/resources/golden/join_filters-46-268d8fb3cb9b04eb269fe7ec40a24dfe similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f rename to sql/hive/src/test/resources/golden/join_filters-46-268d8fb3cb9b04eb269fe7ec40a24dfe diff --git a/sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 b/sql/hive/src/test/resources/golden/join_filters-47-6dc6866a65c74d69538b776b41b06c16 similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 rename to sql/hive/src/test/resources/golden/join_filters-47-6dc6866a65c74d69538b776b41b06c16 diff --git a/sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 b/sql/hive/src/test/resources/golden/join_filters-48-e884480a0f7273d3e2f2de2ba46b855c similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 rename to sql/hive/src/test/resources/golden/join_filters-48-e884480a0f7273d3e2f2de2ba46b855c diff --git a/sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 b/sql/hive/src/test/resources/golden/join_filters-49-98fd86aea9cacaa82d43c7468109dd33 similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 rename to sql/hive/src/test/resources/golden/join_filters-49-98fd86aea9cacaa82d43c7468109dd33 diff --git a/sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e b/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e +++ b/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 b/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 +++ b/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac b/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac +++ b/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 +++ b/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 b/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 +++ b/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba b/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba +++ b/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 b/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 +++ b/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 b/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 +++ b/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba b/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba +++ b/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 b/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 +++ b/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 b/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 +++ b/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 b/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 +++ b/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 b/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 +++ b/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 b/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 +++ b/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e b/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e +++ b/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a b/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a +++ b/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 b/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 +++ b/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada b/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada +++ b/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 b/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 +++ b/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 b/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 +++ b/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 b/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 +++ b/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 b/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 +++ b/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d b/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d +++ b/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e b/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e +++ b/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 b/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 +++ b/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d b/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d +++ b/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 b/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 +++ b/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa b/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa +++ b/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 b/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 +++ b/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 b/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 +++ b/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e b/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e +++ b/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 b/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 +++ b/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf b/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf +++ b/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 b/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 +++ b/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 b/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 +++ b/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 b/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 +++ b/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 b/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 +++ b/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 b/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 +++ b/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f b/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f +++ b/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf b/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf +++ b/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e b/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e +++ b/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e b/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e +++ b/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f b/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f index ea646edf6d143..04ecda59b21bd 100644 --- a/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f +++ b/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL NULL NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 b/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 +++ b/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa b/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa +++ b/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 b/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 +++ b/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca b/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca index ea646edf6d143..04ecda59b21bd 100644 --- a/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca +++ b/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL NULL NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 b/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 +++ b/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 b/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 +++ b/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 b/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 +++ b/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf b/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf +++ b/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f b/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f +++ b/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab b/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab +++ b/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 b/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 +++ b/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 b/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 +++ b/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce b/sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 b/sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 b/sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 b/sql/hive/src/test/resources/golden/join_hive_626-3-4a2f2f2858540afea9a195b5322941ee similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 rename to sql/hive/src/test/resources/golden/join_hive_626-3-4a2f2f2858540afea9a195b5322941ee diff --git a/sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 b/sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80 b/sql/hive/src/test/resources/golden/join_hive_626-4-4bb73b33747da4ed852df381b7b45a71 similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80 rename to sql/hive/src/test/resources/golden/join_hive_626-4-4bb73b33747da4ed852df381b7b45a71 diff --git a/sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 b/sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 b/sql/hive/src/test/resources/golden/join_hive_626-5-c6a3ae6f3539ab48b996060fb51d8ebe similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 rename to sql/hive/src/test/resources/golden/join_hive_626-5-c6a3ae6f3539ab48b996060fb51d8ebe diff --git a/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 b/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 index e3143e49c2bad..8b00ef084005b 100644 --- a/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 +++ b/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 @@ -1 +1 @@ -foo1 bar10 2 \ No newline at end of file +foo1 bar10 2 diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa index 4c88927e56579..120056ea10c60 100644 --- a/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa @@ -104,4 +104,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa index 4c88927e56579..120056ea10c60 100644 --- a/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -104,4 +104,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645 b/sql/hive/src/test/resources/golden/join_nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645 rename to sql/hive/src/test/resources/golden/join_nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f diff --git a/sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 b/sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b b/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b index 4bc2d3969f17f..610f43b2ac6dc 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b +++ b/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b @@ -1,3 +1,3 @@ NULL 35 NULL 35 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 b/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 index 8414cfbede40a..19621e45f3758 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 +++ b/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 b/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 index c11ae6cce7abb..b512581d48dfe 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 +++ b/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d b/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d index e28e1b1e5e8b1..ee5b5e1ec3096 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d +++ b/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d @@ -1,3 +1,3 @@ NULL NULL NULL 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 b/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 index 85192b86801e2..9a862c2d08e7c 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 +++ b/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 @@ -1,3 +1,3 @@ NULL NULL NULL 35 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 b/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 index c19237d5d4309..5d0c8eb89e421 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 +++ b/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 @@ -1,3 +1,3 @@ NULL NULL 48 NULL NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 b/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 index e28e1b1e5e8b1..ee5b5e1ec3096 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 +++ b/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 @@ -1,3 +1,3 @@ NULL NULL NULL 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f b/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f index 56ca29ffe263c..25db723b06378 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f +++ b/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f @@ -2,4 +2,4 @@ NULL NULL NULL 35 NULL NULL 48 NULL NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 b/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 index 92e228d381628..810287110dc40 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 +++ b/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 @@ -1,4 +1,4 @@ NULL NULL NULL 35 NULL 35 NULL NULL 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c b/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c index f57bbd5152852..115f8d9531800 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c +++ b/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c @@ -1,4 +1,4 @@ NULL NULL 48 NULL NULL 35 NULL 35 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 b/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 index 0d14bf4e38509..bb1fee6e62e99 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 +++ b/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 @@ -6,4 +6,4 @@ NULL 35 100 100 48 NULL 100 100 100 100 NULL 35 100 100 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 b/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 index 56ca29ffe263c..25db723b06378 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 +++ b/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 @@ -2,4 +2,4 @@ NULL NULL NULL 35 NULL NULL 48 NULL NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e b/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e index 0db9ffd61a3ad..9bca4244a2eb4 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e +++ b/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e @@ -1,3 +1,3 @@ NULL NULL NULL NULL 48 NULL NULL 35 NULL 35 NULL 35 -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 b/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 index ed1b35fbe80ae..f7ff69b919598 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 +++ b/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 @@ -1,3 +1,3 @@ NULL NULL 48 NULL NULL NULL NULL 35 NULL 35 NULL 35 -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 b/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 index 0db9ffd61a3ad..9bca4244a2eb4 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 +++ b/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 @@ -1,3 +1,3 @@ NULL NULL NULL NULL 48 NULL NULL 35 NULL 35 NULL 35 -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 b/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 index 0d14bf4e38509..bb1fee6e62e99 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 +++ b/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 @@ -6,4 +6,4 @@ NULL 35 100 100 48 NULL 100 100 100 100 NULL 35 100 100 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 b/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 +++ b/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 b/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 index bdd09da460bbe..309eec4c7d309 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 +++ b/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 @@ -1,2 +1,2 @@ 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 b/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 index dfe4f4318fc66..96e8fb0241578 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 +++ b/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 @@ -1,2 +1,2 @@ NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb b/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb +++ b/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb b/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb index bdd09da460bbe..309eec4c7d309 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb +++ b/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb @@ -1,2 +1,2 @@ 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 b/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 index 0d14bf4e38509..bb1fee6e62e99 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 +++ b/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 @@ -6,4 +6,4 @@ NULL 35 100 100 48 NULL 100 100 100 100 NULL 35 100 100 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 b/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 index dfe4f4318fc66..96e8fb0241578 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 +++ b/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 @@ -1,2 +1,2 @@ NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 b/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 +++ b/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 b/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 index c11ae6cce7abb..b512581d48dfe 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 +++ b/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 b/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 index 8414cfbede40a..19621e45f3758 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 +++ b/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 b/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 index 4bc2d3969f17f..610f43b2ac6dc 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 +++ b/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 @@ -1,3 +1,3 @@ NULL 35 NULL 35 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 b/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 index e28e1b1e5e8b1..ee5b5e1ec3096 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 +++ b/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 @@ -1,3 +1,3 @@ NULL NULL NULL 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 b/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 index 85192b86801e2..9a862c2d08e7c 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 +++ b/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 @@ -1,3 +1,3 @@ NULL NULL NULL 35 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc b/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc index c19237d5d4309..5d0c8eb89e421 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc +++ b/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc @@ -1,3 +1,3 @@ NULL NULL 48 NULL NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 b/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 index 0d14bf4e38509..bb1fee6e62e99 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 +++ b/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 @@ -6,4 +6,4 @@ NULL 35 100 100 48 NULL 100 100 100 100 NULL 35 100 100 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 b/sql/hive/src/test/resources/golden/join_nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 rename to sql/hive/src/test/resources/golden/join_nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe diff --git a/sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e b/sql/hive/src/test/resources/golden/join_nulls-41-6dc6866a65c74d69538b776b41b06c16 similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e rename to sql/hive/src/test/resources/golden/join_nulls-41-6dc6866a65c74d69538b776b41b06c16 diff --git a/sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe b/sql/hive/src/test/resources/golden/join_nulls-42-e884480a0f7273d3e2f2de2ba46b855c similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe rename to sql/hive/src/test/resources/golden/join_nulls-42-e884480a0f7273d3e2f2de2ba46b855c diff --git a/sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 b/sql/hive/src/test/resources/golden/join_nulls-43-98fd86aea9cacaa82d43c7468109dd33 similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 rename to sql/hive/src/test/resources/golden/join_nulls-43-98fd86aea9cacaa82d43c7468109dd33 diff --git a/sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 b/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 +++ b/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 b/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 +++ b/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 +++ b/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 b/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 index 7d9efd15fa287..c823f539ee328 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 +++ b/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 @@ -1,4 +1,4 @@ 48 NULL 48 NULL 100 100 100 100 148 NULL 148 NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 b/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 +++ b/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 b/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 index 4d2ff6a237416..2b06d1c29d984 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 +++ b/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 @@ -3,4 +3,4 @@ NULL NULL NULL 135 48 NULL 48 NULL 100 100 100 100 148 NULL 148 NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c b/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c +++ b/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 b/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 index 7d9efd15fa287..c823f539ee328 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 +++ b/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 @@ -1,4 +1,4 @@ 48 NULL 48 NULL 100 100 100 100 148 NULL 148 NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f b/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f index 695bb1544ec89..06f37e109247c 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f +++ b/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL 48 NULL 100 100 100 100 148 NULL 148 NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 b/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 +++ b/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 b/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 +++ b/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 b/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 +++ b/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 b/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 +++ b/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 b/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 index 86b3dab6bffc8..88b1f5331b84e 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 +++ b/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 @@ -1,4 +1,4 @@ NULL 35 NULL 35 NULL 135 NULL 135 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 b/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 index 169df31887b96..b060f40ffdf48 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 +++ b/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 @@ -3,4 +3,4 @@ NULL NULL 148 NULL NULL 35 NULL 35 NULL 135 NULL 135 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a b/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a index 86b3dab6bffc8..88b1f5331b84e 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a +++ b/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a @@ -1,4 +1,4 @@ NULL 35 NULL 35 NULL 135 NULL 135 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 b/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 index bdd5dcf63fd00..1256e9451914b 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 +++ b/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 @@ -3,4 +3,4 @@ NULL 135 NULL 135 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 b/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 index bdd09da460bbe..309eec4c7d309 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 +++ b/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 @@ -1,2 +1,2 @@ 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 b/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 index dfe4f4318fc66..96e8fb0241578 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 +++ b/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 @@ -1,2 +1,2 @@ NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 b/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 +++ b/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b b/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b index c11ae6cce7abb..b512581d48dfe 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b +++ b/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 b/sql/hive/src/test/resources/golden/join_nullsafe-2-5bb63fafa390b1d4c20e225a8a648dcf similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 rename to sql/hive/src/test/resources/golden/join_nullsafe-2-5bb63fafa390b1d4c20e225a8a648dcf diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 b/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645 b/sql/hive/src/test/resources/golden/join_nullsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0 similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645 rename to sql/hive/src/test/resources/golden/join_nullsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 b/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_nullsafe-21-24332b9390108fb3379e1acc599293a1 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/join_nullsafe-21-24332b9390108fb3379e1acc599293a1 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e b/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 b/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_nullsafe-22-4be80634a6bd916e3ebd60a124f0a48e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/join_nullsafe-22-4be80634a6bd916e3ebd60a124f0a48e diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 b/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/join_nullsafe-23-e4425d56be43c21124d95160653ce0ac similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/join_nullsafe-23-e4425d56be43c21124d95160653ce0ac diff --git a/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 b/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 +++ b/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 b/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 index 06a83e95d4f79..33e6a1546ca65 100644 --- a/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 +++ b/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 @@ -1 +1 @@ -2 12 2 22 2 12 2 12 \ No newline at end of file +2 12 2 22 2 12 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder2-4-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/join_reorder2-4-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder2-5-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/join_reorder2-5-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_reorder2-6-8eb53fb8f05a43ee377aa1c927857e7c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/join_reorder2-6-8eb53fb8f05a43ee377aa1c927857e7c diff --git a/sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec b/sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_reorder2-7-5f4cfbbe53c5e808ee08b26514272034 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/join_reorder2-7-5f4cfbbe53c5e808ee08b26514272034 diff --git a/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 b/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 index 06a83e95d4f79..33e6a1546ca65 100644 --- a/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 +++ b/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 @@ -1 +1 @@ -2 12 2 22 2 12 2 12 \ No newline at end of file +2 12 2 22 2 12 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 b/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 index 06a83e95d4f79..33e6a1546ca65 100644 --- a/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 +++ b/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 @@ -1 +1 @@ -2 12 2 22 2 12 2 12 \ No newline at end of file +2 12 2 22 2 12 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/join_reorder3-4-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/join_reorder3-4-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder3-5-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/join_reorder3-5-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_reorder3-6-8eb53fb8f05a43ee377aa1c927857e7c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/join_reorder3-6-8eb53fb8f05a43ee377aa1c927857e7c diff --git a/sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec b/sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/join_reorder3-7-5f4cfbbe53c5e808ee08b26514272034 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/join_reorder3-7-5f4cfbbe53c5e808ee08b26514272034 diff --git a/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 b/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 index 06a83e95d4f79..33e6a1546ca65 100644 --- a/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 +++ b/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 @@ -1 +1 @@ -2 12 2 22 2 12 2 12 \ No newline at end of file +2 12 2 22 2 12 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f b/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f index 872146532307a..19304c010452e 100644 --- a/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f +++ b/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f @@ -1 +1 @@ -2 12 2 22 2 12 \ No newline at end of file +2 12 2 22 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 b/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 index 872146532307a..19304c010452e 100644 --- a/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 +++ b/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 @@ -1 +1 @@ -2 12 2 22 2 12 \ No newline at end of file +2 12 2 22 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/join_reorder4-3-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/join_reorder4-3-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/join_reorder4-4-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/join_reorder4-4-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 b/sql/hive/src/test/resources/golden/join_reorder4-5-8eb53fb8f05a43ee377aa1c927857e7c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 rename to sql/hive/src/test/resources/golden/join_reorder4-5-8eb53fb8f05a43ee377aa1c927857e7c diff --git a/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 b/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 index 872146532307a..19304c010452e 100644 --- a/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 +++ b/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 @@ -1 +1 @@ -2 12 2 22 2 12 \ No newline at end of file +2 12 2 22 2 12 diff --git a/sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e b/sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 b/sql/hive/src/test/resources/golden/join_star-10-a9e579038e3d4826fdae475d7058ab82 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 rename to sql/hive/src/test/resources/golden/join_star-10-a9e579038e3d4826fdae475d7058ab82 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 b/sql/hive/src/test/resources/golden/join_star-11-72730ecdad9c0fd4c6ce64a0cb89fb74 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 rename to sql/hive/src/test/resources/golden/join_star-11-72730ecdad9c0fd4c6ce64a0cb89fb74 diff --git a/sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae b/sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 b/sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 b/sql/hive/src/test/resources/golden/join_star-12-f581d6d305d652cd0f4e4fa912eb578d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 rename to sql/hive/src/test/resources/golden/join_star-12-f581d6d305d652cd0f4e4fa912eb578d diff --git a/sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb b/sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 b/sql/hive/src/test/resources/golden/join_star-13-7268564732cbb7489248f9d818f80c14 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 rename to sql/hive/src/test/resources/golden/join_star-13-7268564732cbb7489248f9d818f80c14 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 b/sql/hive/src/test/resources/golden/join_star-14-2ee0fcf000f8687fc8941bf212477e57 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 rename to sql/hive/src/test/resources/golden/join_star-14-2ee0fcf000f8687fc8941bf212477e57 diff --git a/sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 b/sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 b/sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 b/sql/hive/src/test/resources/golden/join_star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 rename to sql/hive/src/test/resources/golden/join_star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9 diff --git a/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 b/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 +++ b/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 b/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 index e60bca6269264..f97ee71bc2333 100644 --- a/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 +++ b/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 @@ -7,4 +7,4 @@ 61 62 3 71 72 3 81 82 3 -91 92 3 \ No newline at end of file +91 92 3 diff --git a/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b b/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b index e08d04997c56b..e56cb5c03eaa4 100644 --- a/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b +++ b/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b @@ -7,4 +7,4 @@ 61 62 3 4 71 72 3 4 81 82 3 4 -91 92 3 4 \ No newline at end of file +91 92 3 4 diff --git a/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 b/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 index b89911131bd13..80924380b7d76 100644 --- a/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 +++ b/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 @@ -7,4 +7,4 @@ 61 62 3 3 71 72 3 3 81 82 3 3 -91 92 3 3 \ No newline at end of file +91 92 3 3 diff --git a/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 b/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 index b89911131bd13..80924380b7d76 100644 --- a/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 +++ b/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 @@ -7,4 +7,4 @@ 61 62 3 3 71 72 3 3 81 82 3 3 -91 92 3 3 \ No newline at end of file +91 92 3 3 diff --git a/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 b/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 index 19611e75e33c3..ebfbb5cdae331 100644 --- a/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 +++ b/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 @@ -7,4 +7,4 @@ 61 62 3 3 4 4 4 4 4 71 72 3 3 4 4 4 4 4 81 82 3 3 4 4 4 4 4 -91 92 3 3 4 4 4 4 4 \ No newline at end of file +91 92 3 3 4 4 4 4 4 diff --git a/sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 b/sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_star-8-c3d53a4daab9614a09870dc8e9571f74 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/join_star-8-c3d53a4daab9614a09870dc8e9571f74 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_star-9-3f7ccccc2488de5f33a38cb3cc3eb628 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/join_star-9-3f7ccccc2488de5f33a38cb3cc3eb628 diff --git a/sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 b/sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 b/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 deleted file mode 100644 index 07433297e6de1..0000000000000 --- a/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 +++ /dev/null @@ -1,6 +0,0 @@ -aint int from deserializer -astring string from deserializer -lint array from deserializer -lstring array from deserializer -lintstring array> from deserializer -mstringstring map from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db b/sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b b/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b deleted file mode 100644 index 4aeaf4ff1f1b0..0000000000000 --- a/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b +++ /dev/null @@ -1,11 +0,0 @@ -1712634731 [{"myint":0,"mystring":"0","underscore_int":0}] -465985200 [{"myint":1,"mystring":"1","underscore_int":1}] --751827638 [{"myint":4,"mystring":"8","underscore_int":2}] -477111222 [{"myint":9,"mystring":"27","underscore_int":3}] --734328909 [{"myint":16,"mystring":"64","underscore_int":4}] --1952710710 [{"myint":25,"mystring":"125","underscore_int":5}] -1244525190 [{"myint":36,"mystring":"216","underscore_int":6}] --1461153973 [{"myint":49,"mystring":"343","underscore_int":7}] -1638581578 [{"myint":64,"mystring":"512","underscore_int":8}] -336964413 [{"myint":81,"mystring":"729","underscore_int":9}] -0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 b/sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 b/sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 b/sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b b/sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 b/sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef b/sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 b/sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 b/sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 b/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 deleted file mode 100644 index 0da0d93886e01..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 +++ /dev/null @@ -1,2 +0,0 @@ -key1 100 -key2 200 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a b/sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 b/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 deleted file mode 100644 index 0da0d93886e01..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 +++ /dev/null @@ -1,2 +0,0 @@ -key1 100 -key2 200 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe b/sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 b/sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 b/sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa b/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa deleted file mode 100644 index a2a7fdd233a2a..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa +++ /dev/null @@ -1,10 +0,0 @@ -238 val_238 NULL -86 val_86 NULL -311 val_311 NULL -27 val_27 NULL -165 val_165 NULL -409 val_409 NULL -255 val_255 NULL -278 val_278 NULL -98 val_98 NULL -484 val_484 NULL diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 b/sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d b/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d deleted file mode 100644 index 37d918a386d7d..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d +++ /dev/null @@ -1,10 +0,0 @@ -238 val_238 4 -238 val_238 5 -86 val_86 4 -86 val_86 5 -311 val_311 4 -311 val_311 5 -27 val_27 4 -27 val_27 5 -165 val_165 4 -165 val_165 5 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e b/sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 b/sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 b/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 deleted file mode 100644 index 3ad56f646ed85..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 +++ /dev/null @@ -1,10 +0,0 @@ -238 NULL NULL -86 NULL NULL -311 ["val_311","val_311"] val_311 -311 ["val_311","val_311"] val_311 -27 NULL NULL -165 NULL NULL -409 ["val_409","val_409"] val_409 -409 ["val_409","val_409"] val_409 -255 NULL NULL -278 NULL NULL diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b b/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b new file mode 100644 index 0000000000000..cbeb8081f06c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b @@ -0,0 +1,6 @@ +val_0 2 +val_0 3 +val_0 2 +val_0 3 +val_0 2 +val_0 3 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/lateral_view_ppd-9-dc6fea663d875b082d38bd326d21cd95 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/lateral_view_ppd-9-dc6fea663d875b082d38bd326d21cd95 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-5-9c307c0559d735960ce77efa95b2b17b b/sql/hive/src/test/resources/golden/leftsemijoin-5-9c307c0559d735960ce77efa95b2b17b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/leftsemijoin-5-aba449db0d4fe6dc9771426e102bb543 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/leftsemijoin-5-aba449db0d4fe6dc9771426e102bb543 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-6-82921fc96eef547ec0f71027ee88298c b/sql/hive/src/test/resources/golden/leftsemijoin-6-82921fc96eef547ec0f71027ee88298c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/leftsemijoin-6-9f50dce576b019c0be997055b8876621 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/leftsemijoin-6-9f50dce576b019c0be997055b8876621 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 b/sql/hive/src/test/resources/golden/leftsemijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/leftsemijoin-7-fff6ca40e6048d52dc2d3afc68e8353e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/leftsemijoin-7-fff6ca40e6048d52dc2d3afc68e8353e diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-5b2e555868faa404ea09928936178181 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/leftsemijoin_mr-1-5b2e555868faa404ea09928936178181 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-aa3f07f028027ffd13ab5535dc821593 b/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-aa3f07f028027ffd13ab5535dc821593 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 b/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-c019cb2a855138da0d0b1e5c67cd6354 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/leftsemijoin_mr-3-c019cb2a855138da0d0b1e5c67cd6354 diff --git a/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 b/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 index bf0d87ab1b2b0..b8626c4cff284 100644 --- a/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 +++ b/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 @@ -1 +1 @@ -4 \ No newline at end of file +4 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb b/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 b/sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f b/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f deleted file mode 100644 index 9e3b31ad52c13..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f +++ /dev/null @@ -1,20 +0,0 @@ -0 -2 -4 -5 -8 -9 -10 -11 -12 -15 -17 -18 -19 -20 -24 -26 -27 -28 -30 -33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 b/sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 b/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 deleted file mode 100644 index ebf6c0424c26d..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 +++ /dev/null @@ -1,20 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 b/sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 b/sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 b/sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 b/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 deleted file mode 100644 index 153dcec21bc5b..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 +++ /dev/null @@ -1,20 +0,0 @@ -val_0 0 -val_2 2 -val_4 4 -val_8 8 -val_9 9 -val_10 10 -val_11 11 -val_5 15 -val_17 17 -val_19 19 -val_20 20 -val_12 24 -val_27 27 -val_28 28 -val_30 30 -val_15 30 -val_33 33 -val_34 34 -val_18 36 -val_41 41 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 b/sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 b/sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb b/sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 b/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 deleted file mode 100644 index ae8f0265b71ca..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 +++ /dev/null @@ -1,20 +0,0 @@ -val_0 0 -val_10 10 -val_100 200 -val_103 206 -val_104 208 -val_105 105 -val_11 11 -val_111 111 -val_113 226 -val_114 114 -val_116 116 -val_118 236 -val_119 357 -val_12 24 -val_120 240 -val_125 250 -val_126 126 -val_128 384 -val_129 258 -val_131 131 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e b/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a b/sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 b/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 deleted file mode 100644 index e604892422d59..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 +++ /dev/null @@ -1,100 +0,0 @@ -0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 -0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 -0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 -2 val_2 val_2 val_2 val_2 val_2 val_2 val_2 val_2 -4 val_4 val_4 val_4 val_4 val_4 val_4 val_4 val_4 -5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 -5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 -5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 -8 val_8 val_8 val_8 val_8 val_8 val_8 val_8 val_8 -9 val_9 val_9 val_9 val_9 val_9 val_9 val_9 val_9 -10 val_10 val_10 val_10 val_10 val_10 val_10 val_10 val_10 -11 val_11 val_11 val_11 val_11 val_11 val_11 val_11 val_11 -12 val_12 val_12 val_12 val_12 val_12 val_12 val_12 val_12 -12 val_12 val_12 val_12 val_12 val_12 val_12 val_12 val_12 -15 val_15 val_15 val_15 val_15 val_15 val_15 val_15 val_15 -15 val_15 val_15 val_15 val_15 val_15 val_15 val_15 val_15 -17 val_17 val_17 val_17 val_17 val_17 val_17 val_17 val_17 -18 val_18 val_18 val_18 val_18 val_18 val_18 val_18 val_18 -18 val_18 val_18 val_18 val_18 val_18 val_18 val_18 val_18 -19 val_19 val_19 val_19 val_19 val_19 val_19 val_19 val_19 -20 val_20 val_20 val_20 val_20 val_20 val_20 val_20 val_20 -24 val_24 val_24 val_24 val_24 val_24 val_24 val_24 val_24 -24 val_24 val_24 val_24 val_24 val_24 val_24 val_24 val_24 -26 val_26 val_26 val_26 val_26 val_26 val_26 val_26 val_26 -26 val_26 val_26 val_26 val_26 val_26 val_26 val_26 val_26 -27 val_27 val_27 val_27 val_27 val_27 val_27 val_27 val_27 -28 val_28 val_28 val_28 val_28 val_28 val_28 val_28 val_28 -30 val_30 val_30 val_30 val_30 val_30 val_30 val_30 val_30 -33 val_33 val_33 val_33 val_33 val_33 val_33 val_33 val_33 -34 val_34 val_34 val_34 val_34 val_34 val_34 val_34 val_34 -35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 -35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 -35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 -37 val_37 val_37 val_37 val_37 val_37 val_37 val_37 val_37 -37 val_37 val_37 val_37 val_37 val_37 val_37 val_37 val_37 -41 val_41 val_41 val_41 val_41 val_41 val_41 val_41 val_41 -42 val_42 val_42 val_42 val_42 val_42 val_42 val_42 val_42 -42 val_42 val_42 val_42 val_42 val_42 val_42 val_42 val_42 -43 val_43 val_43 val_43 val_43 val_43 val_43 val_43 val_43 -44 val_44 val_44 val_44 val_44 val_44 val_44 val_44 val_44 -47 val_47 val_47 val_47 val_47 val_47 val_47 val_47 val_47 -51 val_51 val_51 val_51 val_51 val_51 val_51 val_51 val_51 -51 val_51 val_51 val_51 val_51 val_51 val_51 val_51 val_51 -53 val_53 val_53 val_53 val_53 val_53 val_53 val_53 val_53 -54 val_54 val_54 val_54 val_54 val_54 val_54 val_54 val_54 -57 val_57 val_57 val_57 val_57 val_57 val_57 val_57 val_57 -58 val_58 val_58 val_58 val_58 val_58 val_58 val_58 val_58 -58 val_58 val_58 val_58 val_58 val_58 val_58 val_58 val_58 -64 val_64 val_64 val_64 val_64 val_64 val_64 val_64 val_64 -65 val_65 val_65 val_65 val_65 val_65 val_65 val_65 val_65 -66 val_66 val_66 val_66 val_66 val_66 val_66 val_66 val_66 -67 val_67 val_67 val_67 val_67 val_67 val_67 val_67 val_67 -67 val_67 val_67 val_67 val_67 val_67 val_67 val_67 val_67 -69 val_69 val_69 val_69 val_69 val_69 val_69 val_69 val_69 -70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 -70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 -70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 -72 val_72 val_72 val_72 val_72 val_72 val_72 val_72 val_72 -72 val_72 val_72 val_72 val_72 val_72 val_72 val_72 val_72 -74 val_74 val_74 val_74 val_74 val_74 val_74 val_74 val_74 -76 val_76 val_76 val_76 val_76 val_76 val_76 val_76 val_76 -76 val_76 val_76 val_76 val_76 val_76 val_76 val_76 val_76 -77 val_77 val_77 val_77 val_77 val_77 val_77 val_77 val_77 -78 val_78 val_78 val_78 val_78 val_78 val_78 val_78 val_78 -80 val_80 val_80 val_80 val_80 val_80 val_80 val_80 val_80 -82 val_82 val_82 val_82 val_82 val_82 val_82 val_82 val_82 -83 val_83 val_83 val_83 val_83 val_83 val_83 val_83 val_83 -83 val_83 val_83 val_83 val_83 val_83 val_83 val_83 val_83 -84 val_84 val_84 val_84 val_84 val_84 val_84 val_84 val_84 -84 val_84 val_84 val_84 val_84 val_84 val_84 val_84 val_84 -85 val_85 val_85 val_85 val_85 val_85 val_85 val_85 val_85 -86 val_86 val_86 val_86 val_86 val_86 val_86 val_86 val_86 -87 val_87 val_87 val_87 val_87 val_87 val_87 val_87 val_87 -90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 -90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 -90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 -92 val_92 val_92 val_92 val_92 val_92 val_92 val_92 val_92 -95 val_95 val_95 val_95 val_95 val_95 val_95 val_95 val_95 -95 val_95 val_95 val_95 val_95 val_95 val_95 val_95 val_95 -96 val_96 val_96 val_96 val_96 val_96 val_96 val_96 val_96 -97 val_97 val_97 val_97 val_97 val_97 val_97 val_97 val_97 -97 val_97 val_97 val_97 val_97 val_97 val_97 val_97 val_97 -98 val_98 val_98 val_98 val_98 val_98 val_98 val_98 val_98 -98 val_98 val_98 val_98 val_98 val_98 val_98 val_98 val_98 -100 val_100 val_100 val_100 val_100 val_100 val_100 val_100 val_100 -100 val_100 val_100 val_100 val_100 val_100 val_100 val_100 val_100 -103 val_103 val_103 val_103 val_103 val_103 val_103 val_103 val_103 -103 val_103 val_103 val_103 val_103 val_103 val_103 val_103 val_103 -104 val_104 val_104 val_104 val_104 val_104 val_104 val_104 val_104 -104 val_104 val_104 val_104 val_104 val_104 val_104 val_104 val_104 -105 val_105 val_105 val_105 val_105 val_105 val_105 val_105 val_105 -111 val_111 val_111 val_111 val_111 val_111 val_111 val_111 val_111 -113 val_113 val_113 val_113 val_113 val_113 val_113 val_113 val_113 -113 val_113 val_113 val_113 val_113 val_113 val_113 val_113 val_113 -114 val_114 val_114 val_114 val_114 val_114 val_114 val_114 val_114 -116 val_116 val_116 val_116 val_116 val_116 val_116 val_116 val_116 -118 val_118 val_118 val_118 val_118 val_118 val_118 val_118 val_118 -118 val_118 val_118 val_118 val_118 val_118 val_118 val_118 val_118 -119 val_119 val_119 val_119 val_119 val_119 val_119 val_119 val_119 -119 val_119 val_119 val_119 val_119 val_119 val_119 val_119 val_119 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a b/sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c b/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c deleted file mode 100644 index 92dc6ce9dbf9e..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c +++ /dev/null @@ -1,100 +0,0 @@ -0 -200 -206 -208 -105 -10 -111 -226 -114 -116 -236 -357 -11 -240 -250 -126 -384 -258 -24 -131 -133 -268 -136 -274 -552 -143 -145 -292 -298 -150 -304 -153 -155 -156 -157 -158 -30 -160 -162 -163 -328 -330 -166 -501 -168 -676 -170 -344 -348 -350 -352 -177 -178 -358 -17 -180 -181 -183 -186 -561 -189 -36 -190 -382 -192 -579 -194 -390 -196 -394 -597 -19 -400 -201 -202 -406 -410 -414 -624 -418 -20 -426 -214 -432 -434 -218 -438 -442 -222 -446 -448 -226 -228 -458 -1150 -466 -235 -474 -476 -478 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 b/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 deleted file mode 100644 index 95f5492558a9b..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 +++ /dev/null @@ -1,20 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 -5 val_5 -5 val_5 -5 val_5 -8 val_8 -9 val_9 -10 val_10 -11 val_11 -12 val_12 -12 val_12 -15 val_15 -15 val_15 -17 val_17 -18 val_18 -18 val_18 -19 val_19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f b/sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 b/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 deleted file mode 100644 index 0bac402cfa497..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 +++ /dev/null @@ -1,20 +0,0 @@ -498 val_498 -498 val_498 -498 val_498 -497 val_497 -496 val_496 -495 val_495 -494 val_494 -493 val_493 -492 val_492 -492 val_492 -491 val_491 -490 val_490 -489 val_489 -489 val_489 -489 val_489 -489 val_489 -487 val_487 -485 val_485 -484 val_484 -483 val_483 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 b/sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 b/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 deleted file mode 100644 index b57d0cc951566..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 +++ /dev/null @@ -1,20 +0,0 @@ -val_0 3 -val_10 11 -val_100 202 -val_103 208 -val_104 210 -val_105 106 -val_11 12 -val_111 112 -val_113 228 -val_114 115 -val_116 117 -val_118 238 -val_119 360 -val_12 26 -val_120 242 -val_125 252 -val_126 127 -val_128 387 -val_129 260 -val_131 132 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 b/sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a b/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a deleted file mode 100644 index 0e95c446f3b0d..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a +++ /dev/null @@ -1,20 +0,0 @@ -val_0 1.0 -val_10 11.0 -val_100 101.0 -val_103 104.0 -val_104 105.0 -val_105 106.0 -val_11 12.0 -val_111 112.0 -val_113 114.0 -val_114 115.0 -val_116 117.0 -val_118 119.0 -val_119 120.0 -val_12 13.0 -val_120 121.0 -val_125 126.0 -val_126 127.0 -val_128 129.0 -val_129 130.0 -val_131 132.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f b/sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/literal_double-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/literal_double-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/literal_double-1-10ef1098e35d900983be3814de8f974f similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/literal_double-1-10ef1098e35d900983be3814de8f974f diff --git a/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 b/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 deleted file mode 100644 index 69505e73feb1d..0000000000000 --- a/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 +++ /dev/null @@ -1 +0,0 @@ -3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 b/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 new file mode 100644 index 0000000000000..24ca45210038a --- /dev/null +++ b/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 @@ -0,0 +1 @@ +3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/literal_ints-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/literal_ints-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 b/sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/literal_ints-1-5ffd1b49cdda4149aef2c61c53a56890 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/literal_ints-1-5ffd1b49cdda4149aef2c61c53a56890 diff --git a/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 b/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 deleted file mode 100644 index 6dc85dd550540..0000000000000 --- a/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 +++ /dev/null @@ -1 +0,0 @@ -100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 b/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 new file mode 100644 index 0000000000000..46a059f07aca6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 @@ -0,0 +1 @@ +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/literal_string-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/literal_string-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 b/sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d b/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d deleted file mode 100644 index 6323f4efa99c7..0000000000000 --- a/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d +++ /dev/null @@ -1 +0,0 @@ -facebook facebook facebook facebook facebook facebook facebook facebook facebook facebook \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/literal_string-1-9b48d41e5c9e41ddc070e2fd31ace15 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/literal_string-1-9b48d41e5c9e41ddc070e2fd31ace15 diff --git a/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d b/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d new file mode 100644 index 0000000000000..1d05317d62547 --- /dev/null +++ b/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d @@ -0,0 +1 @@ +facebook facebook facebook facebook facebook facebook facebook facebook facebook facebook diff --git a/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 b/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc b/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 b/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 b/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca b/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca deleted file mode 100644 index de5212a3c320f772b6a3a2b246202cd99d07c012..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 114 zcmXYpK@xx<31zZO;W_B|0{Q<~&`YzM%(UUWxDW~|hJHUCips`;RC1i>>>>>>>>>>>>>>>>>>>>>> None -max_nested_map array>>>>>>>>>>>>>>>>>>>>> None -max_nested_struct array>>>>>>>>>>>>>>>>>>>>>> None -simple_string string None diff --git a/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 b/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 deleted file mode 100644 index 8ce70d8c71782..0000000000000 --- a/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 +++ /dev/null @@ -1,8 +0,0 @@ -simple_int int None -max_nested_array array>>>>>>>>>>>>>>>>>>>>>> None -max_nested_map array>>>>>>>>>>>>>>>>>>>>> None -max_nested_struct array>>>>>>>>>>>>>>>>>>>>>> None -simple_string string None - -Detailed Table Information Table(tableName:nestedcomplex, dbName:default, owner:marmbrus, createTime:1391226936, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:simple_int, type:int, comment:null), FieldSchema(name:max_nested_array, type:array>>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:max_nested_map, type:array>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:max_nested_struct, type:array>>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:simple_string, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/nestedcomplex, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1, line.delim= -, hive.serialization.extend.nesting.levels=true}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1391226936}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 b/sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d b/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d deleted file mode 100644 index 228853cffe527..0000000000000 --- a/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d +++ /dev/null @@ -1,2 +0,0 @@ -2 [[[[[[[[[[[[[[[[[[[[[[[0,3,2]]]]]]]]]]]]]]]]]]]]]]] [[[[[[[[[[[[[[[[[[[[[{"k1":"v1","k3":"v3"}]]]]]]]]]]]]]]]]]]]]] [[[[[[[[[[[[[[[[[[[[[[{"s":"b","i":10}]]]]]]]]]]]]]]]]]]]]]] 2 -3 [[[[[[[[[[[[[[[[[[[[[[[0,1,2]]]]]]]]]]]]]]]]]]]]]]] [[[[[[[[[[[[[[[[[[[[[{"k1":"v1","k2":"v2"}]]]]]]]]]]]]]]]]]]]]] [[[[[[[[[[[[[[[[[[[[[[{"s":"a","i":10}]]]]]]]]]]]]]]]]]]]]]] 2 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d b/sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 b/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 index cdcd12dca3ca2..9c4a8fd131407 100644 --- a/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 +++ b/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 @@ -81,4 +81,4 @@ val_84 val_28 val_37 val_90 -val_97 \ No newline at end of file +val_97 diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 index a0cfcf2621f2e..34e8c1127f016 100644 --- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 @@ -22,4 +22,4 @@ NULL val_484 25 311 val_311 25 369 25 401 val_401 25 -406 val_406 25 \ No newline at end of file +406 val_406 25 diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 index a0cfcf2621f2e..34e8c1127f016 100644 --- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 @@ -22,4 +22,4 @@ NULL val_484 25 311 val_311 25 369 25 401 val_401 25 -406 val_406 25 \ No newline at end of file +406 val_406 25 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 b/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 b/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 deleted file mode 100644 index 24e19ec6caa8f..0000000000000 --- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 +++ /dev/null @@ -1 +0,0 @@ --826625916 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 deleted file mode 100644 index 05b0c43f926bd..0000000000000 --- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 +++ /dev/null @@ -1 +0,0 @@ -10226524244 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 deleted file mode 100644 index bc15b9ca54985..0000000000000 --- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 +++ /dev/null @@ -1 +0,0 @@ -20453048488 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad deleted file mode 100644 index 08839f6bb296e..0000000000000 --- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad +++ /dev/null @@ -1 +0,0 @@ -200 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f index fce122031e88f..138522d542895 100644 --- a/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f @@ -54,4 +54,4 @@ 1234 95 2.0 1234 96 1.0 1234 97 2.0 -1234 98 2.0 \ No newline at end of file +1234 98 2.0 diff --git a/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f index fce122031e88f..138522d542895 100644 --- a/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f @@ -54,4 +54,4 @@ 1234 95 2.0 1234 96 1.0 1234 97 2.0 -1234 98 2.0 \ No newline at end of file +1234 98 2.0 diff --git a/sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 b/sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c b/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c deleted file mode 100644 index d111428eaabb0..0000000000000 --- a/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c +++ /dev/null @@ -1 +0,0 @@ -[null,0] [null,[]] [null,{}] [null,{"col1":0}] diff --git a/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 +++ b/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 +++ b/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 +++ b/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 +++ b/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/nullgroup3-1-61ead7f73d525e0d9e21beba91a3d39e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/nullgroup3-1-61ead7f73d525e0d9e21beba91a3d39e diff --git a/sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 b/sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/nullgroup3-12-61ead7f73d525e0d9e21beba91a3d39e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/nullgroup3-12-61ead7f73d525e0d9e21beba91a3d39e diff --git a/sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 b/sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/nullgroup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb b/sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 b/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 +++ b/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-18-113e2b587784d54d2a5b5f091389397e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/nullgroup3-18-113e2b587784d54d2a5b5f091389397e diff --git a/sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 b/sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-19-77de4b2d65eee228848625cdd422178d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/nullgroup3-19-77de4b2d65eee228848625cdd422178d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 b/sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-2-a5e12cfbc1799ce9fa9628d81b8c0b06 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/nullgroup3-2-a5e12cfbc1799ce9fa9628d81b8c0b06 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb b/sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 b/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 +++ b/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 b/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 +++ b/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-6-113e2b587784d54d2a5b5f091389397e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/nullgroup3-6-113e2b587784d54d2a5b5f091389397e diff --git a/sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 b/sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-7-77de4b2d65eee228848625cdd422178d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/nullgroup3-7-77de4b2d65eee228848625cdd422178d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 b/sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 b/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 +++ b/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup5-1-642e12a05bf01a6716bfa15ed0012629 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/nullgroup5-1-642e12a05bf01a6716bfa15ed0012629 diff --git a/sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b b/sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup5-3-35517978e09aa1bd1d15f51d11e08fd5 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/nullgroup5-3-35517978e09aa1bd1d15f51d11e08fd5 diff --git a/sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c b/sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c b/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c +++ b/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 b/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 +++ b/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 b/sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/nullscript-1-f07dfd6ef687e038083deca5941d8174 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/nullscript-1-f07dfd6ef687e038083deca5941d8174 diff --git a/sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 b/sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/nullscript-2-bb1abcf2f4a2a5cd5c058104901627bb similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/nullscript-2-bb1abcf2f4a2a5cd5c058104901627bb diff --git a/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 b/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 index d23e05acf7ba5..e34118512c1d7 100644 --- a/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 +++ b/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 @@ -497,4 +497,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc b/sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 b/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 deleted file mode 100644 index da0ddb96957e1..0000000000000 --- a/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL 1 0 0.0 diff --git a/sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 b/sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d b/sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 deleted file mode 100644 index 7541739b48608..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 b/sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 deleted file mode 100644 index 78c0010bebbc6..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 +++ /dev/null @@ -1,34 +0,0 @@ -# col_name data_type comment - -key int from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create -Table Type: MANAGED_TABLE -Table Parameters: - last_modified_by marmbrus - last_modified_time 1391226987 - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde -InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d b/sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 deleted file mode 100644 index 6881640dd3cd4..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde -InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 b/sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 b/sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 b/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 deleted file mode 100644 index dd4fa77f326ad..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 +++ /dev/null @@ -1,29 +0,0 @@ -# col_name data_type comment - -str string from deserializer -mp map from deserializer -lst array from deserializer -strct struct from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create_complex -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde -InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 b/sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 b/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 deleted file mode 100644 index 88c8812029d82..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 +++ /dev/null @@ -1,3 +0,0 @@ -line1 {"key11":"value11","key12":"value12","key13":"value13"} ["a","b","c"] {"a":"one","b":"two"} -line2 {"key21":"value21","key22":"value22","key23":"value23"} ["d","e","f"] {"a":"three","b":"four"} -line3 {"key31":"value31","key32":"value32","key33":"value33"} ["g","h","i"] {"a":"five","b":"six"} diff --git a/sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 b/sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 b/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 deleted file mode 100644 index 67946888f1baf..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 +++ /dev/null @@ -1,3 +0,0 @@ -line1 {"key12":"value12","key11":"value11","key13":"value13"} ["a","b","c"] {"A":"one","B":"two"} -line2 {"key21":"value21","key23":"value23","key22":"value22"} ["d","e","f"] {"A":"three","B":"four"} -line3 {"key33":"value33","key31":"value31","key32":"value32"} ["g","h","i"] {"A":"five","B":"six"} diff --git a/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead b/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead deleted file mode 100644 index 83db48f84ec87..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead +++ /dev/null @@ -1,3 +0,0 @@ -line1 -line2 -line3 diff --git a/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c b/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c deleted file mode 100644 index db4876dd3c809..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c +++ /dev/null @@ -1,3 +0,0 @@ -{"key12":"value12","key11":"value11","key13":"value13"} -{"key21":"value21","key23":"value23","key22":"value22"} -{"key33":"value33","key31":"value31","key32":"value32"} diff --git a/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 b/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 deleted file mode 100644 index 3b2e1cf7e0098..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 +++ /dev/null @@ -1,3 +0,0 @@ -["a","b","c"] -["d","e","f"] -["g","h","i"] diff --git a/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 b/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 deleted file mode 100644 index 0f890dfd226d6..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 +++ /dev/null @@ -1,3 +0,0 @@ -{"a":"one","b":"two"} -{"a":"three","b":"four"} -{"a":"five","b":"six"} diff --git a/sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 b/sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae b/sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d b/sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 b/sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 b/sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a b/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a deleted file mode 100644 index 0c6f532488607..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a +++ /dev/null @@ -1,10 +0,0 @@ -91 Genevieve Wilkins 908 Turpis. Street -93 Mariko Cline P.O. Box 329, 5375 Ac St. -95 Winifred Hopper Ap #140-8982 Velit Avenue -97 Dana Carter 814-601 Purus. Av. -99 Wynter Vincent 626-8492 Mollis Avenue -92 Thane Oneil 6766 Lectus St. -94 Lael Mclean 500-7010 Sit St. -96 Rafael England P.O. Box 405, 7857 Eget Av. -98 Juliet Battle Ap #535-1965 Cursus St. -100 Wang Mitchell 4023 Lacinia. Ave diff --git a/sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 b/sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 b/sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae b/sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 b/sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 b/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 deleted file mode 100644 index ecd0704ce3386..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -str string None -mp map None -lst array None -strct struct None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create_staging -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226986 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - colelction.delim , - field.delim | - mapkey.delim : - serialization.format | diff --git a/sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 b/sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 deleted file mode 100644 index 6881640dd3cd4..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde -InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d index f799fca27c353..5bba4beaf3355 100644 --- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d @@ -7,4 +7,4 @@ 255 278 98 -484 \ No newline at end of file +484 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 index f953881c894c1..0b6d77d2e825b 100644 --- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 @@ -1 +1 @@ -1082202951192 \ No newline at end of file +1082202951192 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e b/sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 deleted file mode 100644 index f172be3e72712..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 +++ /dev/null @@ -1,10 +0,0 @@ -0 3 2 -165 NULL 1 -2 1 2 -238 NULL 1 -27 NULL 1 -311 NULL 1 -4 1 2 -5 3 2 -8 1 2 -86 NULL 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 b/sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf b/sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 b/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 deleted file mode 100644 index e3c0074c9d4f6..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 +++ /dev/null @@ -1,9 +0,0 @@ - - - - - - - - - diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 b/sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e b/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e deleted file mode 100644 index 4cd1242d9fe61..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e +++ /dev/null @@ -1,10 +0,0 @@ - - - - - - - -165 -255 -27 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f deleted file mode 100644 index e7700b8aee5c6..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f +++ /dev/null @@ -1,5 +0,0 @@ -1 -NULL -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 b/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 index 217a1915f8826..4dba25230ed1b 100644 --- a/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 +++ b/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 @@ -7,4 +7,4 @@ 5 val_5 5 val_5 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 b/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 index ab34c87e42364..fde2ef3e83ce5 100644 --- a/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 +++ b/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 @@ -7,4 +7,4 @@ 494 val_494 493 val_493 492 val_492 -492 val_492 \ No newline at end of file +492 val_492 diff --git a/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f b/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f index 217a1915f8826..4dba25230ed1b 100644 --- a/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f +++ b/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f @@ -7,4 +7,4 @@ 5 val_5 5 val_5 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 b/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 index 549da558b18da..83a26e92a694b 100644 --- a/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 +++ b/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 19 val_19 19 val_19 -19 val_19 19 val_19 \ No newline at end of file +19 val_19 19 val_19 diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 b/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 index 549da558b18da..83a26e92a694b 100644 --- a/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 +++ b/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 19 val_19 19 val_19 -19 val_19 19 val_19 \ No newline at end of file +19 val_19 19 val_19 diff --git a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 +++ b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 b/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 +++ b/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a b/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a index c912af4a5c676..5710fb29e7e02 100644 --- a/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a +++ b/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a @@ -306,4 +306,4 @@ 495 val_495 496 val_496 497 val_497 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 b/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 index c912af4a5c676..5710fb29e7e02 100644 --- a/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 +++ b/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 @@ -306,4 +306,4 @@ 495 val_495 496 val_496 497 val_497 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a b/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a index c912af4a5c676..5710fb29e7e02 100644 --- a/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a +++ b/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a @@ -306,4 +306,4 @@ 495 val_495 496 val_496 497 val_497 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 b/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 index c912af4a5c676..5710fb29e7e02 100644 --- a/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 +++ b/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 @@ -306,4 +306,4 @@ 495 val_495 496 val_496 497 val_497 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c +++ b/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d b/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 b/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 deleted file mode 100644 index b431d3fc6dcf6..0000000000000 --- a/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string None -value string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Apr 29 20:55:07 PDT 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 0 - numRows 48 - rawDataSize 512 - totalSize 560 - transient_lastDdlTime 1398830107 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 deleted file mode 100644 index 3b733e2d6c451..0000000000000 --- a/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 +++ /dev/null @@ -1,48 +0,0 @@ -128 val_128 -128 val_128 -150 val_150 -150 val_150 -165 val_165 -165 val_165 -193 val_193 -193 val_193 -213 val_213 -213 val_213 -213 val_213 -213 val_213 -213 val_214 -213 val_214 -224 val_224 -224 val_224 -238 val_238 -238 val_238 -238 val_239 -238 val_239 -238 val_240 -238 val_240 -255 val_255 -255 val_255 -265 val_265 -265 val_265 -27 val_27 -27 val_27 -273 val_273 -273 val_273 -278 val_278 -278 val_278 -311 val_311 -311 val_311 -369 val_369 -369 val_369 -401 val_401 -401 val_401 -409 val_409 -409 val_409 -484 val_484 -484 val_484 -66 val_66 -66 val_66 -86 val_86 -86 val_86 -98 val_98 -98 val_98 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f b/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 deleted file mode 100644 index 7f67251e61787..0000000000000 --- a/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string None -value string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Apr 29 20:54:55 PDT 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 0 - numRows 48 - rawDataSize 512 - totalSize 560 - transient_lastDdlTime 1398830095 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 deleted file mode 100644 index 3b733e2d6c451..0000000000000 --- a/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 +++ /dev/null @@ -1,48 +0,0 @@ -128 val_128 -128 val_128 -150 val_150 -150 val_150 -165 val_165 -165 val_165 -193 val_193 -193 val_193 -213 val_213 -213 val_213 -213 val_213 -213 val_213 -213 val_214 -213 val_214 -224 val_224 -224 val_224 -238 val_238 -238 val_238 -238 val_239 -238 val_239 -238 val_240 -238 val_240 -255 val_255 -255 val_255 -265 val_265 -265 val_265 -27 val_27 -27 val_27 -273 val_273 -273 val_273 -278 val_278 -278 val_278 -311 val_311 -311 val_311 -369 val_369 -369 val_369 -401 val_401 -401 val_401 -409 val_409 -409 val_409 -484 val_484 -484 val_484 -66 val_66 -66 val_66 -86 val_86 -86 val_86 -98 val_98 -98 val_98 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 b/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 b/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc b/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd b/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec b/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 b/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 b/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea b/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 b/sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b b/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 b/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 b/sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 b/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 index ffa0de8a0c322..a98dc8e8e1b86 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -1,24 +1,24 @@ # col_name data_type comment -c1 tinyint None +c1 tinyint # Partition Information # col_name data_type comment -c2 string None +c2 string # Detailed Partition Information Partition Value: [v1] Database: default Table: mytbl -CreateTime: Fri Jan 03 17:18:52 PST 2014 +CreateTime: Tue Oct 21 04:00:26 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1 Partition Parameters: a myval b yourval - transient_lastDdlTime 1388798332 + transient_lastDdlTime 1413889226 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -29,4 +29,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 index 0909bab11a6ec..2bef99dafeb29 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -1,22 +1,22 @@ # col_name data_type comment -c1 tinyint None +c1 tinyint # Partition Information # col_name data_type comment -c2 string None +c2 string # Detailed Partition Information Partition Value: [v1] Database: default Table: mytbl -CreateTime: Fri Jan 03 17:18:59 PST 2014 +CreateTime: Tue Oct 21 04:00:27 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1 Partition Parameters: - transient_lastDdlTime 1388798339 + transient_lastDdlTime 1413889227 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -27,4 +27,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 index 55fb6df62f0a7..7b6b90f7f4200 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -1,25 +1,25 @@ # col_name data_type comment -c1 tinyint None +c1 tinyint # Partition Information # col_name data_type comment -c2 string None +c2 string # Detailed Partition Information Partition Value: [v1] Database: default Table: mytbl -CreateTime: Fri Jan 03 17:19:02 PST 2014 +CreateTime: Tue Oct 21 04:00:28 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1 Partition Parameters: a myval b yourval c noval - transient_lastDdlTime 1388798342 + transient_lastDdlTime 1413889228 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -30,4 +30,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 b/sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partcols1-1-b562ff3e36de23a686d122967a1f91c8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/partcols1-1-b562ff3e36de23a686d122967a1f91c8 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_date-1-6b0952309c3ebdd6dcb7066891d1bd74 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/partition_date-1-6b0952309c3ebdd6dcb7066891d1bd74 diff --git a/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f b/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 b/sql/hive/src/test/resources/golden/partition_date-11-a01c3791f59e819c750e213b6c65d084 similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 rename to sql/hive/src/test/resources/golden/partition_date-11-a01c3791f59e819c750e213b6c65d084 diff --git a/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 b/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 new file mode 100644 index 0000000000000..b4de394767536 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 @@ -0,0 +1 @@ +11 diff --git a/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 b/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 new file mode 100644 index 0000000000000..64bb6b746dcea --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 @@ -0,0 +1 @@ +30 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/partition_date-14-3f187dad9a2fdfc6f7a3566309016f9c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/partition_date-14-3f187dad9a2fdfc6f7a3566309016f9c diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/partition_date-15-e4366325f3a0c4a8e92be59f4de73fce similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/partition_date-15-e4366325f3a0c4a8e92be59f4de73fce diff --git a/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac b/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 b/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 b/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d b/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_date-2-220048240a7050a98411ddbc6b1f82cf similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/partition_date-2-220048240a7050a98411ddbc6b1f82cf diff --git a/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba b/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 b/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 b/sql/hive/src/test/resources/golden/partition_date-22-fe277bd0a30e016826d242fd5bd61714 similarity index 100% rename from sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 rename to sql/hive/src/test/resources/golden/partition_date-22-fe277bd0a30e016826d242fd5bd61714 diff --git a/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 b/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_date-24-7ec1f3a845e2c49191460e15af30aa30 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/partition_date-24-7ec1f3a845e2c49191460e15af30aa30 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_date-3-f8887dd18c21bf0306e293d463b3e1d7 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/partition_date-3-f8887dd18c21bf0306e293d463b3e1d7 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partition_date-4-f7c20c208e052305c1710a5e1e80c5c8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/partition_date-4-f7c20c208e052305c1710a5e1e80c5c8 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_date-5-59355f4e222dcd4f77f51d15bd896f11 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/partition_date-5-59355f4e222dcd4f77f51d15bd896f11 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_date-6-416ab10ac818d432cd3a81870d6e5164 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/partition_date-6-416ab10ac818d432cd3a81870d6e5164 diff --git a/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc b/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc new file mode 100644 index 0000000000000..758a01e987473 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc @@ -0,0 +1,3 @@ +2000-01-01 +2013-08-08 +2013-12-10 diff --git a/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 b/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 new file mode 100644 index 0000000000000..24192eefd2caf --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 @@ -0,0 +1,5 @@ +165 val_165 2000-01-01 2 +238 val_238 2000-01-01 2 +27 val_27 2000-01-01 2 +311 val_311 2000-01-01 2 +86 val_86 2000-01-01 2 diff --git a/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 b/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 b/sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 b/sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 deleted file mode 100644 index 81df179618406..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 +++ /dev/null @@ -1,3 +0,0 @@ -ts=2011-01-11+14:18:26 -ts=2011-01-11+15:18:26 -ts=2011-01-11+16:18:26 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 deleted file mode 100644 index e16053e3110ae..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 +++ /dev/null @@ -1,3 +0,0 @@ -ts=2011-01-11+14%3A18%3A26 -ts=2011-01-11+15%3A18%3A26 -ts=2011-01-11+16%3A18%3A26 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 b/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 index 8af58e2b29f4a..c97e50a8a58cd 100644 --- a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 +++ b/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 @@ -1,8 +1,8 @@ -key string None -value string None -dt string None +key string +value string +dt string # Partition Information # col_name data_type comment -dt string None \ No newline at end of file +dt string diff --git a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 b/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 index 82a07522b0af0..39db984c884ad 100644 --- a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 +++ b/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 @@ -1,9 +1,9 @@ -key string None -value string None -x string None -dt string None +key string +value string +x string +dt string # Partition Information # col_name data_type comment -dt string None \ No newline at end of file +dt string diff --git a/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 b/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 index 8af58e2b29f4a..c97e50a8a58cd 100644 --- a/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 +++ b/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 @@ -1,8 +1,8 @@ -key string None -value string None -dt string None +key string +value string +dt string # Partition Information # col_name data_type comment -dt string None \ No newline at end of file +dt string diff --git a/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 b/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 index 95f5492558a9b..7e53c64af58aa 100644 --- a/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 +++ b/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 @@ -17,4 +17,4 @@ 17 val_17 18 val_18 18 val_18 -19 val_19 \ No newline at end of file +19 val_19 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 b/sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 b/sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 deleted file mode 100644 index e16053e3110ae..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 +++ /dev/null @@ -1,3 +0,0 @@ -ts=2011-01-11+14%3A18%3A26 -ts=2011-01-11+15%3A18%3A26 -ts=2011-01-11+16%3A18%3A26 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 deleted file mode 100644 index e16053e3110ae..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 +++ /dev/null @@ -1,3 +0,0 @@ -ts=2011-01-11+14%3A18%3A26 -ts=2011-01-11+15%3A18%3A26 -ts=2011-01-11+16%3A18%3A26 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_type_check-11-60aad884be613d18d3b89fca3b90dc94 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/partition_type_check-11-60aad884be613d18d3b89fca3b90dc94 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a b/sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_type_check-2-3a6bb204d9524299f28adf1cc35d6f4d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/partition_type_check-2-3a6bb204d9524299f28adf1cc35d6f4d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f b/sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 b/sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partition_type_check-6-663051c7106b8ee6913ca1b007157941 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/partition_type_check-6-663051c7106b8ee6913ca1b007157941 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_varchar1-2-bca5c9edccc3a84e0a9ef92ebcbe746 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/partition_varchar1-2-bca5c9edccc3a84e0a9ef92ebcbe746 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 b/sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 b/sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_varchar1-3-9cec4b1d156f5a9cb587470b98de15 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/partition_varchar1-3-9cec4b1d156f5a9cb587470b98de15 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef b/sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_varchar1-4-e9c4530e270db6d44cc54292e4eff680 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/partition_varchar1-4-e9c4530e270db6d44cc54292e4eff680 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_varchar1-5-661aefd18c44c1eb02c2aaf8fab59f73 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/partition_varchar1-5-661aefd18c44c1eb02c2aaf8fab59f73 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce b/sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde deleted file mode 100644 index 39c80f1b77eab..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde +++ /dev/null @@ -1,1000 +0,0 @@ -0 val_0 NULL -0 val_0 NULL -0 val_0 NULL -0 val_0 0 -0 val_0 0 -0 val_0 0 -10 val_10 NULL -10 val_10 10 -100 val_100 NULL -100 val_100 NULL -100 val_100 100 -100 val_100 100 -103 val_103 NULL -103 val_103 NULL -103 val_103 103 -103 val_103 103 -104 val_104 NULL -104 val_104 NULL -104 val_104 104 -104 val_104 104 -105 val_105 NULL -105 val_105 105 -11 val_11 NULL -11 val_11 11 -111 val_111 NULL -111 val_111 111 -113 val_113 NULL -113 val_113 NULL -113 val_113 113 -113 val_113 113 -114 val_114 NULL -114 val_114 114 -116 val_116 NULL -116 val_116 116 -118 val_118 NULL -118 val_118 NULL -118 val_118 118 -118 val_118 118 -119 val_119 NULL -119 val_119 NULL -119 val_119 NULL -119 val_119 119 -119 val_119 119 -119 val_119 119 -12 val_12 NULL -12 val_12 NULL -12 val_12 12 -12 val_12 12 -120 val_120 NULL -120 val_120 NULL -120 val_120 120 -120 val_120 120 -125 val_125 NULL -125 val_125 NULL -125 val_125 125 -125 val_125 125 -126 val_126 NULL -126 val_126 126 -128 val_128 NULL -128 val_128 NULL -128 val_128 NULL -128 val_128 128 -128 val_128 128 -128 val_128 128 -129 val_129 NULL -129 val_129 NULL -129 val_129 129 -129 val_129 129 -131 val_131 NULL -131 val_131 131 -133 val_133 NULL -133 val_133 133 -134 val_134 NULL -134 val_134 NULL -134 val_134 134 -134 val_134 134 -136 val_136 NULL -136 val_136 136 -137 val_137 NULL -137 val_137 NULL -137 val_137 137 -137 val_137 137 -138 val_138 NULL -138 val_138 NULL -138 val_138 NULL -138 val_138 NULL -138 val_138 138 -138 val_138 138 -138 val_138 138 -138 val_138 138 -143 val_143 NULL -143 val_143 143 -145 val_145 NULL -145 val_145 145 -146 val_146 NULL -146 val_146 NULL -146 val_146 146 -146 val_146 146 -149 val_149 NULL -149 val_149 NULL -149 val_149 149 -149 val_149 149 -15 val_15 NULL -15 val_15 NULL -15 val_15 15 -15 val_15 15 -150 val_150 NULL -150 val_150 150 -152 val_152 NULL -152 val_152 NULL -152 val_152 152 -152 val_152 152 -153 val_153 NULL -153 val_153 153 -155 val_155 NULL -155 val_155 155 -156 val_156 NULL -156 val_156 156 -157 val_157 NULL -157 val_157 157 -158 val_158 NULL -158 val_158 158 -160 val_160 NULL -160 val_160 160 -162 val_162 NULL -162 val_162 162 -163 val_163 NULL -163 val_163 163 -164 val_164 NULL -164 val_164 NULL -164 val_164 164 -164 val_164 164 -165 val_165 NULL -165 val_165 NULL -165 val_165 165 -165 val_165 165 -166 val_166 NULL -166 val_166 166 -167 val_167 NULL -167 val_167 NULL -167 val_167 NULL -167 val_167 167 -167 val_167 167 -167 val_167 167 -168 val_168 NULL -168 val_168 168 -169 val_169 NULL -169 val_169 NULL -169 val_169 NULL -169 val_169 NULL -169 val_169 169 -169 val_169 169 -169 val_169 169 -169 val_169 169 -17 val_17 NULL -17 val_17 17 -170 val_170 NULL -170 val_170 170 -172 val_172 NULL -172 val_172 NULL -172 val_172 172 -172 val_172 172 -174 val_174 NULL -174 val_174 NULL -174 val_174 174 -174 val_174 174 -175 val_175 NULL -175 val_175 NULL -175 val_175 175 -175 val_175 175 -176 val_176 NULL -176 val_176 NULL -176 val_176 176 -176 val_176 176 -177 val_177 NULL -177 val_177 177 -178 val_178 NULL -178 val_178 178 -179 val_179 NULL -179 val_179 NULL -179 val_179 179 -179 val_179 179 -18 val_18 NULL -18 val_18 NULL -18 val_18 18 -18 val_18 18 -180 val_180 NULL -180 val_180 180 -181 val_181 NULL -181 val_181 181 -183 val_183 NULL -183 val_183 183 -186 val_186 NULL -186 val_186 186 -187 val_187 NULL -187 val_187 NULL -187 val_187 NULL -187 val_187 187 -187 val_187 187 -187 val_187 187 -189 val_189 NULL -189 val_189 189 -19 val_19 NULL -19 val_19 19 -190 val_190 NULL -190 val_190 190 -191 val_191 NULL -191 val_191 NULL -191 val_191 191 -191 val_191 191 -192 val_192 NULL -192 val_192 192 -193 val_193 NULL -193 val_193 NULL -193 val_193 NULL -193 val_193 193 -193 val_193 193 -193 val_193 193 -194 val_194 NULL -194 val_194 194 -195 val_195 NULL -195 val_195 NULL -195 val_195 195 -195 val_195 195 -196 val_196 NULL -196 val_196 196 -197 val_197 NULL -197 val_197 NULL -197 val_197 197 -197 val_197 197 -199 val_199 NULL -199 val_199 NULL -199 val_199 NULL -199 val_199 199 -199 val_199 199 -199 val_199 199 -2 val_2 NULL -2 val_2 2 -20 val_20 NULL -20 val_20 20 -200 val_200 NULL -200 val_200 NULL -200 val_200 200 -200 val_200 200 -201 val_201 NULL -201 val_201 201 -202 val_202 NULL -202 val_202 202 -203 val_203 NULL -203 val_203 NULL -203 val_203 203 -203 val_203 203 -205 val_205 NULL -205 val_205 NULL -205 val_205 205 -205 val_205 205 -207 val_207 NULL -207 val_207 NULL -207 val_207 207 -207 val_207 207 -208 val_208 NULL -208 val_208 NULL -208 val_208 NULL -208 val_208 208 -208 val_208 208 -208 val_208 208 -209 val_209 NULL -209 val_209 NULL -209 val_209 209 -209 val_209 209 -213 val_213 NULL -213 val_213 NULL -213 val_213 213 -213 val_213 213 -214 val_214 NULL -214 val_214 214 -216 val_216 NULL -216 val_216 NULL -216 val_216 216 -216 val_216 216 -217 val_217 NULL -217 val_217 NULL -217 val_217 217 -217 val_217 217 -218 val_218 NULL -218 val_218 218 -219 val_219 NULL -219 val_219 NULL -219 val_219 219 -219 val_219 219 -221 val_221 NULL -221 val_221 NULL -221 val_221 221 -221 val_221 221 -222 val_222 NULL -222 val_222 222 -223 val_223 NULL -223 val_223 NULL -223 val_223 223 -223 val_223 223 -224 val_224 NULL -224 val_224 NULL -224 val_224 224 -224 val_224 224 -226 val_226 NULL -226 val_226 226 -228 val_228 NULL -228 val_228 228 -229 val_229 NULL -229 val_229 NULL -229 val_229 229 -229 val_229 229 -230 val_230 NULL -230 val_230 NULL -230 val_230 NULL -230 val_230 NULL -230 val_230 NULL -230 val_230 230 -230 val_230 230 -230 val_230 230 -230 val_230 230 -230 val_230 230 -233 val_233 NULL -233 val_233 NULL -233 val_233 233 -233 val_233 233 -235 val_235 NULL -235 val_235 235 -237 val_237 NULL -237 val_237 NULL -237 val_237 237 -237 val_237 237 -238 val_238 NULL -238 val_238 NULL -238 val_238 238 -238 val_238 238 -239 val_239 NULL -239 val_239 NULL -239 val_239 239 -239 val_239 239 -24 val_24 NULL -24 val_24 NULL -24 val_24 24 -24 val_24 24 -241 val_241 NULL -241 val_241 241 -242 val_242 NULL -242 val_242 NULL -242 val_242 242 -242 val_242 242 -244 val_244 NULL -244 val_244 244 -247 val_247 NULL -247 val_247 247 -248 val_248 NULL -248 val_248 248 -249 val_249 NULL -249 val_249 249 -252 val_252 NULL -252 val_252 252 -255 val_255 NULL -255 val_255 NULL -255 val_255 255 -255 val_255 255 -256 val_256 NULL -256 val_256 NULL -256 val_256 256 -256 val_256 256 -257 val_257 NULL -257 val_257 257 -258 val_258 NULL -258 val_258 258 -26 val_26 NULL -26 val_26 NULL -26 val_26 26 -26 val_26 26 -260 val_260 NULL -260 val_260 260 -262 val_262 NULL -262 val_262 262 -263 val_263 NULL -263 val_263 263 -265 val_265 NULL -265 val_265 NULL -265 val_265 265 -265 val_265 265 -266 val_266 NULL -266 val_266 266 -27 val_27 NULL -27 val_27 27 -272 val_272 NULL -272 val_272 NULL -272 val_272 272 -272 val_272 272 -273 val_273 NULL -273 val_273 NULL -273 val_273 NULL -273 val_273 273 -273 val_273 273 -273 val_273 273 -274 val_274 NULL -274 val_274 274 -275 val_275 NULL -275 val_275 275 -277 val_277 NULL -277 val_277 NULL -277 val_277 NULL -277 val_277 NULL -277 val_277 277 -277 val_277 277 -277 val_277 277 -277 val_277 277 -278 val_278 NULL -278 val_278 NULL -278 val_278 278 -278 val_278 278 -28 val_28 NULL -28 val_28 28 -280 val_280 NULL -280 val_280 NULL -280 val_280 280 -280 val_280 280 -281 val_281 NULL -281 val_281 NULL -281 val_281 281 -281 val_281 281 -282 val_282 NULL -282 val_282 NULL -282 val_282 282 -282 val_282 282 -283 val_283 NULL -283 val_283 283 -284 val_284 NULL -284 val_284 284 -285 val_285 NULL -285 val_285 285 -286 val_286 NULL -286 val_286 286 -287 val_287 NULL -287 val_287 287 -288 val_288 NULL -288 val_288 NULL -288 val_288 288 -288 val_288 288 -289 val_289 NULL -289 val_289 289 -291 val_291 NULL -291 val_291 291 -292 val_292 NULL -292 val_292 292 -296 val_296 NULL -296 val_296 296 -298 val_298 NULL -298 val_298 NULL -298 val_298 NULL -298 val_298 298 -298 val_298 298 -298 val_298 298 -30 val_30 NULL -30 val_30 30 -302 val_302 NULL -302 val_302 302 -305 val_305 NULL -305 val_305 305 -306 val_306 NULL -306 val_306 306 -307 val_307 NULL -307 val_307 NULL -307 val_307 307 -307 val_307 307 -308 val_308 NULL -308 val_308 308 -309 val_309 NULL -309 val_309 NULL -309 val_309 309 -309 val_309 309 -310 val_310 NULL -310 val_310 310 -311 val_311 NULL -311 val_311 NULL -311 val_311 NULL -311 val_311 311 -311 val_311 311 -311 val_311 311 -315 val_315 NULL -315 val_315 315 -316 val_316 NULL -316 val_316 NULL -316 val_316 NULL -316 val_316 316 -316 val_316 316 -316 val_316 316 -317 val_317 NULL -317 val_317 NULL -317 val_317 317 -317 val_317 317 -318 val_318 NULL -318 val_318 NULL -318 val_318 NULL -318 val_318 318 -318 val_318 318 -318 val_318 318 -321 val_321 NULL -321 val_321 NULL -321 val_321 321 -321 val_321 321 -322 val_322 NULL -322 val_322 NULL -322 val_322 322 -322 val_322 322 -323 val_323 NULL -323 val_323 323 -325 val_325 NULL -325 val_325 NULL -325 val_325 325 -325 val_325 325 -327 val_327 NULL -327 val_327 NULL -327 val_327 NULL -327 val_327 327 -327 val_327 327 -327 val_327 327 -33 val_33 NULL -33 val_33 33 -331 val_331 NULL -331 val_331 NULL -331 val_331 331 -331 val_331 331 -332 val_332 NULL -332 val_332 332 -333 val_333 NULL -333 val_333 NULL -333 val_333 333 -333 val_333 333 -335 val_335 NULL -335 val_335 335 -336 val_336 NULL -336 val_336 336 -338 val_338 NULL -338 val_338 338 -339 val_339 NULL -339 val_339 339 -34 val_34 NULL -34 val_34 34 -341 val_341 NULL -341 val_341 341 -342 val_342 NULL -342 val_342 NULL -342 val_342 342 -342 val_342 342 -344 val_344 NULL -344 val_344 NULL -344 val_344 344 -344 val_344 344 -345 val_345 NULL -345 val_345 345 -348 val_348 NULL -348 val_348 NULL -348 val_348 NULL -348 val_348 NULL -348 val_348 NULL -348 val_348 348 -348 val_348 348 -348 val_348 348 -348 val_348 348 -348 val_348 348 -35 val_35 NULL -35 val_35 NULL -35 val_35 NULL -35 val_35 35 -35 val_35 35 -35 val_35 35 -351 val_351 NULL -351 val_351 351 -353 val_353 NULL -353 val_353 NULL -353 val_353 353 -353 val_353 353 -356 val_356 NULL -356 val_356 356 -360 val_360 NULL -360 val_360 360 -362 val_362 NULL -362 val_362 362 -364 val_364 NULL -364 val_364 364 -365 val_365 NULL -365 val_365 365 -366 val_366 NULL -366 val_366 366 -367 val_367 NULL -367 val_367 NULL -367 val_367 367 -367 val_367 367 -368 val_368 NULL -368 val_368 368 -369 val_369 NULL -369 val_369 NULL -369 val_369 NULL -369 val_369 369 -369 val_369 369 -369 val_369 369 -37 val_37 NULL -37 val_37 NULL -37 val_37 37 -37 val_37 37 -373 val_373 NULL -373 val_373 373 -374 val_374 NULL -374 val_374 374 -375 val_375 NULL -375 val_375 375 -377 val_377 NULL -377 val_377 377 -378 val_378 NULL -378 val_378 378 -379 val_379 NULL -379 val_379 379 -382 val_382 NULL -382 val_382 NULL -382 val_382 382 -382 val_382 382 -384 val_384 NULL -384 val_384 NULL -384 val_384 NULL -384 val_384 384 -384 val_384 384 -384 val_384 384 -386 val_386 NULL -386 val_386 386 -389 val_389 NULL -389 val_389 389 -392 val_392 NULL -392 val_392 392 -393 val_393 NULL -393 val_393 393 -394 val_394 NULL -394 val_394 394 -395 val_395 NULL -395 val_395 NULL -395 val_395 395 -395 val_395 395 -396 val_396 NULL -396 val_396 NULL -396 val_396 NULL -396 val_396 396 -396 val_396 396 -396 val_396 396 -397 val_397 NULL -397 val_397 NULL -397 val_397 397 -397 val_397 397 -399 val_399 NULL -399 val_399 NULL -399 val_399 399 -399 val_399 399 -4 val_4 NULL -4 val_4 4 -400 val_400 NULL -400 val_400 400 -401 val_401 NULL -401 val_401 NULL -401 val_401 NULL -401 val_401 NULL -401 val_401 NULL -401 val_401 401 -401 val_401 401 -401 val_401 401 -401 val_401 401 -401 val_401 401 -402 val_402 NULL -402 val_402 402 -403 val_403 NULL -403 val_403 NULL -403 val_403 NULL -403 val_403 403 -403 val_403 403 -403 val_403 403 -404 val_404 NULL -404 val_404 NULL -404 val_404 404 -404 val_404 404 -406 val_406 NULL -406 val_406 NULL -406 val_406 NULL -406 val_406 NULL -406 val_406 406 -406 val_406 406 -406 val_406 406 -406 val_406 406 -407 val_407 NULL -407 val_407 407 -409 val_409 NULL -409 val_409 NULL -409 val_409 NULL -409 val_409 409 -409 val_409 409 -409 val_409 409 -41 val_41 NULL -41 val_41 41 -411 val_411 NULL -411 val_411 411 -413 val_413 NULL -413 val_413 NULL -413 val_413 413 -413 val_413 413 -414 val_414 NULL -414 val_414 NULL -414 val_414 414 -414 val_414 414 -417 val_417 NULL -417 val_417 NULL -417 val_417 NULL -417 val_417 417 -417 val_417 417 -417 val_417 417 -418 val_418 NULL -418 val_418 418 -419 val_419 NULL -419 val_419 419 -42 val_42 NULL -42 val_42 NULL -42 val_42 42 -42 val_42 42 -421 val_421 NULL -421 val_421 421 -424 val_424 NULL -424 val_424 NULL -424 val_424 424 -424 val_424 424 -427 val_427 NULL -427 val_427 427 -429 val_429 NULL -429 val_429 NULL -429 val_429 429 -429 val_429 429 -43 val_43 NULL -43 val_43 43 -430 val_430 NULL -430 val_430 NULL -430 val_430 NULL -430 val_430 430 -430 val_430 430 -430 val_430 430 -431 val_431 NULL -431 val_431 NULL -431 val_431 NULL -431 val_431 431 -431 val_431 431 -431 val_431 431 -432 val_432 NULL -432 val_432 432 -435 val_435 NULL -435 val_435 435 -436 val_436 NULL -436 val_436 436 -437 val_437 NULL -437 val_437 437 -438 val_438 NULL -438 val_438 NULL -438 val_438 NULL -438 val_438 438 -438 val_438 438 -438 val_438 438 -439 val_439 NULL -439 val_439 NULL -439 val_439 439 -439 val_439 439 -44 val_44 NULL -44 val_44 44 -443 val_443 NULL -443 val_443 443 -444 val_444 NULL -444 val_444 444 -446 val_446 NULL -446 val_446 446 -448 val_448 NULL -448 val_448 448 -449 val_449 NULL -449 val_449 449 -452 val_452 NULL -452 val_452 452 -453 val_453 NULL -453 val_453 453 -454 val_454 NULL -454 val_454 NULL -454 val_454 NULL -454 val_454 454 -454 val_454 454 -454 val_454 454 -455 val_455 NULL -455 val_455 455 -457 val_457 NULL -457 val_457 457 -458 val_458 NULL -458 val_458 NULL -458 val_458 458 -458 val_458 458 -459 val_459 NULL -459 val_459 NULL -459 val_459 459 -459 val_459 459 -460 val_460 NULL -460 val_460 460 -462 val_462 NULL -462 val_462 NULL -462 val_462 462 -462 val_462 462 -463 val_463 NULL -463 val_463 NULL -463 val_463 463 -463 val_463 463 -466 val_466 NULL -466 val_466 NULL -466 val_466 NULL -466 val_466 466 -466 val_466 466 -466 val_466 466 -467 val_467 NULL -467 val_467 467 -468 val_468 NULL -468 val_468 NULL -468 val_468 NULL -468 val_468 NULL -468 val_468 468 -468 val_468 468 -468 val_468 468 -468 val_468 468 -469 val_469 NULL -469 val_469 NULL -469 val_469 NULL -469 val_469 NULL -469 val_469 NULL -469 val_469 469 -469 val_469 469 -469 val_469 469 -469 val_469 469 -469 val_469 469 -47 val_47 NULL -47 val_47 47 -470 val_470 NULL -470 val_470 470 -472 val_472 NULL -472 val_472 472 -475 val_475 NULL -475 val_475 475 -477 val_477 NULL -477 val_477 477 -478 val_478 NULL -478 val_478 NULL -478 val_478 478 -478 val_478 478 -479 val_479 NULL -479 val_479 479 -480 val_480 NULL -480 val_480 NULL -480 val_480 NULL -480 val_480 480 -480 val_480 480 -480 val_480 480 -481 val_481 NULL -481 val_481 481 -482 val_482 NULL -482 val_482 482 -483 val_483 NULL -483 val_483 483 -484 val_484 NULL -484 val_484 484 -485 val_485 NULL -485 val_485 485 -487 val_487 NULL -487 val_487 487 -489 val_489 NULL -489 val_489 NULL -489 val_489 NULL -489 val_489 NULL -489 val_489 489 -489 val_489 489 -489 val_489 489 -489 val_489 489 -490 val_490 NULL -490 val_490 490 -491 val_491 NULL -491 val_491 491 -492 val_492 NULL -492 val_492 NULL -492 val_492 492 -492 val_492 492 -493 val_493 NULL -493 val_493 493 -494 val_494 NULL -494 val_494 494 -495 val_495 NULL -495 val_495 495 -496 val_496 NULL -496 val_496 496 -497 val_497 NULL -497 val_497 497 -498 val_498 NULL -498 val_498 NULL -498 val_498 NULL -498 val_498 498 -498 val_498 498 -498 val_498 498 -5 val_5 NULL -5 val_5 NULL -5 val_5 NULL -5 val_5 5 -5 val_5 5 -5 val_5 5 -51 val_51 NULL -51 val_51 NULL -51 val_51 51 -51 val_51 51 -53 val_53 NULL -53 val_53 53 -54 val_54 NULL -54 val_54 54 -57 val_57 NULL -57 val_57 57 -58 val_58 NULL -58 val_58 NULL -58 val_58 58 -58 val_58 58 -64 val_64 NULL -64 val_64 64 -65 val_65 NULL -65 val_65 65 -66 val_66 NULL -66 val_66 66 -67 val_67 NULL -67 val_67 NULL -67 val_67 67 -67 val_67 67 -69 val_69 NULL -69 val_69 69 -70 val_70 NULL -70 val_70 NULL -70 val_70 NULL -70 val_70 70 -70 val_70 70 -70 val_70 70 -72 val_72 NULL -72 val_72 NULL -72 val_72 72 -72 val_72 72 -74 val_74 NULL -74 val_74 74 -76 val_76 NULL -76 val_76 NULL -76 val_76 76 -76 val_76 76 -77 val_77 NULL -77 val_77 77 -78 val_78 NULL -78 val_78 78 -8 val_8 NULL -8 val_8 8 -80 val_80 NULL -80 val_80 80 -82 val_82 NULL -82 val_82 82 -83 val_83 NULL -83 val_83 NULL -83 val_83 83 -83 val_83 83 -84 val_84 NULL -84 val_84 NULL -84 val_84 84 -84 val_84 84 -85 val_85 NULL -85 val_85 85 -86 val_86 NULL -86 val_86 86 -87 val_87 NULL -87 val_87 87 -9 val_9 NULL -9 val_9 9 -90 val_90 NULL -90 val_90 NULL -90 val_90 NULL -90 val_90 90 -90 val_90 90 -90 val_90 90 -92 val_92 NULL -92 val_92 92 -95 val_95 NULL -95 val_95 NULL -95 val_95 95 -95 val_95 95 -96 val_96 NULL -96 val_96 96 -97 val_97 NULL -97 val_97 NULL -97 val_97 97 -97 val_97 97 -98 val_98 NULL -98 val_98 NULL -98 val_98 98 -98 val_98 98 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a deleted file mode 100644 index dcf40c875b9ad..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=101 -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:284 -maxFileSize:284 -minFileSize:284 -lastAccessTime:0 -lastUpdateTime:1390900729000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 deleted file mode 100644 index 9639a1e84c615..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 +++ /dev/null @@ -1,50 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 deleted file mode 100644 index 707036ebbc76c..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned -inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:3 -totalFileSize:1415 -maxFileSize:895 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900762000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 deleted file mode 100644 index 06316a924c38b..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100 -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:236 -maxFileSize:236 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900762000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a deleted file mode 100644 index 735d41e54e958..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=101 -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:284 -maxFileSize:284 -minFileSize:284 -lastAccessTime:0 -lastUpdateTime:1390900762000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 deleted file mode 100644 index 27dfc622c9bb6..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=102 -inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:895 -maxFileSize:895 -minFileSize:895 -lastAccessTime:0 -lastUpdateTime:1390900762000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 deleted file mode 100644 index 1812e0d53e443..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:236 -maxFileSize:236 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900706000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 deleted file mode 100644 index fca99e91bbd8f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 +++ /dev/null @@ -1,75 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f b/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f deleted file mode 100644 index fca99e91bbd8f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f +++ /dev/null @@ -1,75 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 deleted file mode 100644 index 3532257511613..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100 -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:236 -maxFileSize:236 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900706000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 deleted file mode 100644 index a02458b88bf52..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:2 -totalFileSize:520 -maxFileSize:284 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900729000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 deleted file mode 100644 index 301bff44316ff..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100 -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:236 -maxFileSize:236 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900729000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e deleted file mode 100644 index 001841c8a1cd4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf deleted file mode 100644 index 63f35fd827de3..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf deleted file mode 100644 index 63f35fd827de3..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e deleted file mode 100644 index f75126345f351..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 11542e84a3768..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 53aab40f88b50..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 1b97612ce3f5a..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,4 +0,0 @@ -476 val_238 -476 val_238 -194 val_97 -194 val_97 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index f8e13d5235028..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,4 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 -97 val_97 NULL 2 -97 val_97 NULL 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 deleted file mode 100644 index 919a66a94c5cb..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 +++ /dev/null @@ -1,6 +0,0 @@ -476 val_238 NULL -476 val_238 NULL -194 val_97 NULL -194 val_97 NULL -400 val_200 val_200 -400 val_200 val_200 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 45c169f0d8330..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,6 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 -97 val_97 NULL 2 -97 val_97 NULL 2 -200 val_200 val_200 3 -200 val_200 val_200 3 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 53aab40f88b50..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 deleted file mode 100644 index b6a7d89c68e0c..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 +++ /dev/null @@ -1 +0,0 @@ -16 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d deleted file mode 100644 index b6a7d89c68e0c..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d +++ /dev/null @@ -1 +0,0 @@ -16 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 deleted file mode 100644 index 425151f3a411f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 +++ /dev/null @@ -1 +0,0 @@ -40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 deleted file mode 100644 index 425151f3a411f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 +++ /dev/null @@ -1 +0,0 @@ -40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 deleted file mode 100644 index 425151f3a411f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 +++ /dev/null @@ -1 +0,0 @@ -40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 deleted file mode 100644 index c739b42c4d2ce..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 +++ /dev/null @@ -1 +0,0 @@ -44 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 11542e84a3768..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 deleted file mode 100644 index 025abe4ec330b..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 +++ /dev/null @@ -1,3 +0,0 @@ -476 val_238 NULL 1 -476 val_238 NULL 1 -172 val_86 val_86 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index f067c1ed602dc..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,3 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 -86 val_86 val_86 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 53aab40f88b50..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 11542e84a3768..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 deleted file mode 100644 index 025abe4ec330b..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 +++ /dev/null @@ -1,3 +0,0 @@ -476 val_238 NULL 1 -476 val_238 NULL 1 -172 val_86 val_86 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index f067c1ed602dc..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,3 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 -86 val_86 val_86 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 53aab40f88b50..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 deleted file mode 100644 index 325e26b3d9737..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 +++ /dev/null @@ -1,75 +0,0 @@ -238 val_238 100 -NULL 100 -311 val_311 100 -NULL val_27 100 -NULL val_165 100 -NULL val_409 100 -255 val_255 100 -278 val_278 100 -98 val_98 100 -NULL val_484 100 -NULL val_265 100 -NULL val_193 100 -401 val_401 100 -150 val_150 100 -273 val_273 100 -224 100 -369 100 -66 val_66 100 -128 100 -213 val_213 100 -146 val_146 100 -406 val_406 100 -NULL 100 -NULL 100 -NULL 100 -238 val_238 101 -NULL 101 -311 val_311 101 -NULL val_27 101 -NULL val_165 101 -NULL val_409 101 -255 val_255 101 -278 val_278 101 -98 val_98 101 -NULL val_484 101 -NULL val_265 101 -NULL val_193 101 -401 val_401 101 -150 val_150 101 -273 val_273 101 -224 101 -369 101 -66 val_66 101 -128 101 -213 val_213 101 -146 val_146 101 -406 val_406 101 -NULL 101 -NULL 101 -NULL 101 -238 val_238 102 -NULL 102 -311 val_311 102 -NULL val_27 102 -NULL val_165 102 -NULL val_409 102 -255 val_255 102 -278 val_278 102 -98 val_98 102 -NULL val_484 102 -NULL val_265 102 -NULL val_193 102 -401 val_401 102 -150 val_150 102 -273 val_273 102 -224 102 -369 102 -66 val_66 102 -128 102 -213 val_213 102 -146 val_146 102 -406 val_406 102 -NULL 102 -NULL 102 -NULL 102 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 deleted file mode 100644 index 325e26b3d9737..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 +++ /dev/null @@ -1,75 +0,0 @@ -238 val_238 100 -NULL 100 -311 val_311 100 -NULL val_27 100 -NULL val_165 100 -NULL val_409 100 -255 val_255 100 -278 val_278 100 -98 val_98 100 -NULL val_484 100 -NULL val_265 100 -NULL val_193 100 -401 val_401 100 -150 val_150 100 -273 val_273 100 -224 100 -369 100 -66 val_66 100 -128 100 -213 val_213 100 -146 val_146 100 -406 val_406 100 -NULL 100 -NULL 100 -NULL 100 -238 val_238 101 -NULL 101 -311 val_311 101 -NULL val_27 101 -NULL val_165 101 -NULL val_409 101 -255 val_255 101 -278 val_278 101 -98 val_98 101 -NULL val_484 101 -NULL val_265 101 -NULL val_193 101 -401 val_401 101 -150 val_150 101 -273 val_273 101 -224 101 -369 101 -66 val_66 101 -128 101 -213 val_213 101 -146 val_146 101 -406 val_406 101 -NULL 101 -NULL 101 -NULL 101 -238 val_238 102 -NULL 102 -311 val_311 102 -NULL val_27 102 -NULL val_165 102 -NULL val_409 102 -255 val_255 102 -278 val_278 102 -98 val_98 102 -NULL val_484 102 -NULL val_265 102 -NULL val_193 102 -401 val_401 102 -150 val_150 102 -273 val_273 102 -224 102 -369 102 -66 val_66 102 -128 102 -213 val_213 102 -146 val_146 102 -406 val_406 102 -NULL 102 -NULL 102 -NULL 102 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 deleted file mode 100644 index 79c8f8e614a1f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a deleted file mode 100644 index f487b340cd55b..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a +++ /dev/null @@ -1,14 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=101 -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:284 -maxFileSize:284 -minFileSize:284 -lastAccessTime:0 -lastUpdateTime:1388799388000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 deleted file mode 100644 index 0c7739c2b9fbb..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=102 -inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:895 -maxFileSize:895 -minFileSize:895 -lastAccessTime:0 -lastUpdateTime:1388799405000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a deleted file mode 100644 index 79c8f8e614a1f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a deleted file mode 100644 index 0c6fbc997de7a..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a +++ /dev/null @@ -1,14 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=101 -inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:895 -maxFileSize:895 -minFileSize:895 -lastAccessTime:0 -lastUpdateTime:1388799434000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 index f0d140e18c053..eb4c6a843cb5d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 @@ -1,2 +1,2 @@ 101 25 -102 25 \ No newline at end of file +102 25 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 index 6425bae2a6a8f..95846abf28b2f 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 @@ -1,3 +1,3 @@ 101 25 102 25 -103 25 \ No newline at end of file +103 25 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f index d8263ee986059..0cfbf08886fca 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f @@ -1 +1 @@ -2 \ No newline at end of file +2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 index d8263ee986059..0cfbf08886fca 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 @@ -1 +1 @@ -2 \ No newline at end of file +2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 index 3f10ffe7a4c47..60d3b2f4a4cd5 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 @@ -1 +1 @@ -15 \ No newline at end of file +15 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 index d8263ee986059..0cfbf08886fca 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 @@ -1 +1 @@ -2 \ No newline at end of file +2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b deleted file mode 100644 index 5e06930239fea..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b +++ /dev/null @@ -1,20 +0,0 @@ -0 val_0 1 -0 val_0 1 -0 val_0 1 -0 val_0 2 -0 val_0 2 -0 val_0 2 -0 val_0 3 -0 val_0 3 -0 val_0 3 -10 val_10 1 -10 val_10 2 -10 val_10 3 -100 val_100 1 -100 val_100 1 -100 val_100 2 -100 val_100 2 -100 val_100 3 -100 val_100 3 -103 val_103 1 -103 val_103 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd deleted file mode 100644 index 1bd9063a9c63c..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd +++ /dev/null @@ -1,20 +0,0 @@ -0.0 val_0 1 -0.0 val_0 1 -0.0 val_0 1 -0.0 val_0 2 -0.0 val_0 2 -0.0 val_0 2 -0.0 val_0 3 -0.0 val_0 3 -0.0 val_0 3 -4.0 val_2 1 -4.0 val_2 2 -4.0 val_2 3 -8.0 val_4 1 -8.0 val_4 2 -8.0 val_4 3 -10.0 val_5 1 -10.0 val_5 1 -10.0 val_5 1 -10.0 val_5 2 -10.0 val_5 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b index f259b4eefc608..967305a18236e 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b @@ -17,4 +17,4 @@ 104 val_104 1 104 val_104 1 104 val_104 2 -104 val_104 2 \ No newline at end of file +104 val_104 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd index 89afce150e7a9..f26a7375a6819 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd @@ -17,4 +17,4 @@ 16.0 val_8 1 16.0 val_8 2 18.0 val_9 1 -18.0 val_9 2 \ No newline at end of file +18.0 val_9 2 diff --git a/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 b/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 index 55b2a1b47afa2..ef98fd7908a01 100644 --- a/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 +++ b/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 @@ -493,4 +493,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 b/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 index 55b2a1b47afa2..ef98fd7908a01 100644 --- a/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 +++ b/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 @@ -493,4 +493,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b b/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b index 878c86cb46b8a..d2aff0f701be9 100644 --- a/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b +++ b/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b @@ -133,4 +133,4 @@ 480 3 489 4 492 2 -498 3 \ No newline at end of file +498 3 diff --git a/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b b/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b index 878c86cb46b8a..d2aff0f701be9 100644 --- a/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b +++ b/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b @@ -133,4 +133,4 @@ 480 3 489 4 492 2 -498 3 \ No newline at end of file +498 3 diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c b/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c index c87107a2f1168..416fbfb9e5228 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c @@ -22,4 +22,4 @@ NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL -NULL NULL NULL \ No newline at end of file +NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c b/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c index c87107a2f1168..416fbfb9e5228 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c @@ -22,4 +22,4 @@ NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL -NULL NULL NULL \ No newline at end of file +NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 b/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 index e37d32abba426..83b33d238dab9 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 +++ b/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 @@ -1 +1 @@ -1000 \ No newline at end of file +1000 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d b/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d index e1659e6072577..f005a4fe5bf86 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d +++ b/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d @@ -126,4 +126,4 @@ val_395 val_396 val_397 val_399 -val_4 \ No newline at end of file +val_4 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d b/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d index e1659e6072577..f005a4fe5bf86 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d +++ b/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d @@ -126,4 +126,4 @@ val_395 val_396 val_397 val_399 -val_4 \ No newline at end of file +val_4 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f b/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f index 7e66866e2dd60..ab02a73437ed7 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f +++ b/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f @@ -2,4 +2,4 @@ val_4 1 val_399 2 val_396 3 val_277 4 -val_348 5 \ No newline at end of file +val_348 5 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f b/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f index 7e66866e2dd60..ab02a73437ed7 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f +++ b/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f @@ -2,4 +2,4 @@ val_4 1 val_399 2 val_396 3 val_277 4 -val_348 5 \ No newline at end of file +val_348 5 diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 b/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 index fbce9efa766d7..a29747aef8046 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 +++ b/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 @@ -709,4 +709,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 b/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 index fbce9efa766d7..a29747aef8046 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 +++ b/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 @@ -709,4 +709,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 b/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 index 9e6c4359e78a8..f2748faa4ea7e 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 +++ b/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 @@ -1707,4 +1707,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 b/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 index 9e6c4359e78a8..f2748faa4ea7e 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 +++ b/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 @@ -1707,4 +1707,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b b/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b index 0e11aea14d61d..91f2ce2b61787 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b +++ b/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b @@ -1764,4 +1764,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b b/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b index 0e11aea14d61d..91f2ce2b61787 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b +++ b/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b @@ -1764,4 +1764,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 b/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 index 997f37b76bedb..c0ffb7aeca9b6 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 b/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 index 997f37b76bedb..c0ffb7aeca9b6 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 b/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 index 997f37b76bedb..c0ffb7aeca9b6 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 b/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 index 997f37b76bedb..c0ffb7aeca9b6 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 b/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 index b3bf95dd32bac..9365b77dec065 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -19 val_19 19 val_19 \ No newline at end of file +19 val_19 19 val_19 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 b/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 index b3bf95dd32bac..9365b77dec065 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -19 val_19 19 val_19 \ No newline at end of file +19 val_19 19 val_19 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe b/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe index d4a363c49aeaf..daa4ea309e2c1 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe @@ -7,4 +7,4 @@ 18 val_18 18 val_18 18 18 val_18 18 val_18 18 18 val_18 18 val_18 18 -18 val_18 18 val_18 18 \ No newline at end of file +18 val_18 18 val_18 18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe b/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe index d4a363c49aeaf..daa4ea309e2c1 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe @@ -7,4 +7,4 @@ 18 val_18 18 val_18 18 18 val_18 18 val_18 18 18 val_18 18 val_18 18 -18 val_18 18 val_18 18 \ No newline at end of file +18 val_18 18 val_18 18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 b/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 deleted file mode 100644 index 0190981db84ed..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 +++ /dev/null @@ -1,84 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -10 val_10 -11 val_11 -12 val_12 -12 val_12 -15 val_15 -15 val_15 -17 val_17 -18 val_18 -18 val_18 -19 val_19 -2 val_2 -20 val_20 -24 val_24 -24 val_24 -26 val_26 -26 val_26 -27 val_27 -28 val_28 -30 val_30 -33 val_33 -34 val_34 -35 val_35 -35 val_35 -35 val_35 -37 val_37 -37 val_37 -4 val_4 -41 val_41 -42 val_42 -42 val_42 -43 val_43 -44 val_44 -47 val_47 -5 val_5 -5 val_5 -5 val_5 -51 val_51 -51 val_51 -53 val_53 -54 val_54 -57 val_57 -58 val_58 -58 val_58 -64 val_64 -65 val_65 -66 val_66 -67 val_67 -67 val_67 -69 val_69 -70 val_70 -70 val_70 -70 val_70 -72 val_72 -72 val_72 -74 val_74 -76 val_76 -76 val_76 -77 val_77 -78 val_78 -8 val_8 -80 val_80 -82 val_82 -83 val_83 -83 val_83 -84 val_84 -84 val_84 -85 val_85 -86 val_86 -87 val_87 -9 val_9 -90 val_90 -90 val_90 -90 val_90 -92 val_92 -95 val_95 -95 val_95 -96 val_96 -97 val_97 -97 val_97 -98 val_98 -98 val_98 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 b/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 deleted file mode 100644 index 0190981db84ed..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 +++ /dev/null @@ -1,84 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -10 val_10 -11 val_11 -12 val_12 -12 val_12 -15 val_15 -15 val_15 -17 val_17 -18 val_18 -18 val_18 -19 val_19 -2 val_2 -20 val_20 -24 val_24 -24 val_24 -26 val_26 -26 val_26 -27 val_27 -28 val_28 -30 val_30 -33 val_33 -34 val_34 -35 val_35 -35 val_35 -35 val_35 -37 val_37 -37 val_37 -4 val_4 -41 val_41 -42 val_42 -42 val_42 -43 val_43 -44 val_44 -47 val_47 -5 val_5 -5 val_5 -5 val_5 -51 val_51 -51 val_51 -53 val_53 -54 val_54 -57 val_57 -58 val_58 -58 val_58 -64 val_64 -65 val_65 -66 val_66 -67 val_67 -67 val_67 -69 val_69 -70 val_70 -70 val_70 -70 val_70 -72 val_72 -72 val_72 -74 val_74 -76 val_76 -76 val_76 -77 val_77 -78 val_78 -8 val_8 -80 val_80 -82 val_82 -83 val_83 -83 val_83 -84 val_84 -84 val_84 -85 val_85 -86 val_86 -87 val_87 -9 val_9 -90 val_90 -90 val_90 -90 val_90 -92 val_92 -95 val_95 -95 val_95 -96 val_96 -97 val_97 -97 val_97 -98 val_98 -98 val_98 diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e b/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e index 4c3919232c73e..c30d4a581ba6c 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e +++ b/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e @@ -168,4 +168,4 @@ 90 val_90 403 val_403 400 val_400 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e b/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e index 4c3919232c73e..c30d4a581ba6c 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e +++ b/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e @@ -168,4 +168,4 @@ 90 val_90 403 val_403 400 val_400 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 b/sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 b/sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f b/sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c b/sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 b/sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 b/sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 b/sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 b/sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 b/sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c b/sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 b/sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c b/sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 b/sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 b/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 deleted file mode 100644 index d183a30ddf3b6..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 +++ /dev/null @@ -1,36 +0,0 @@ -238 val_238 2008-04-08 11 -86 val_86 2008-04-08 11 -311 val_311 2008-04-08 11 -27 val_27 2008-04-08 11 -165 val_165 2008-04-08 11 -409 val_409 2008-04-08 11 -255 val_255 2008-04-08 11 -278 val_278 2008-04-08 11 -98 val_98 2008-04-08 11 -238 val_238 2008-04-08 12 -86 val_86 2008-04-08 12 -311 val_311 2008-04-08 12 -27 val_27 2008-04-08 12 -165 val_165 2008-04-08 12 -409 val_409 2008-04-08 12 -255 val_255 2008-04-08 12 -278 val_278 2008-04-08 12 -98 val_98 2008-04-08 12 -238 val_238 2008-04-09 11 -86 val_86 2008-04-09 11 -311 val_311 2008-04-09 11 -27 val_27 2008-04-09 11 -165 val_165 2008-04-09 11 -409 val_409 2008-04-09 11 -255 val_255 2008-04-09 11 -278 val_278 2008-04-09 11 -98 val_98 2008-04-09 11 -238 val_238 2008-04-09 12 -86 val_86 2008-04-09 12 -311 val_311 2008-04-09 12 -27 val_27 2008-04-09 12 -165 val_165 2008-04-09 12 -409 val_409 2008-04-09 12 -255 val_255 2008-04-09 12 -278 val_278 2008-04-09 12 -98 val_98 2008-04-09 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 b/sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 b/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 deleted file mode 100644 index 5b2461e35b5f4..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 +++ /dev/null @@ -1,36 +0,0 @@ -238 val_238 2008-04-08 11 0 -238 val_238 2008-04-08 11 0 -86 val_86 2008-04-08 11 12 -311 val_311 2008-04-08 11 22 -311 val_311 2008-04-08 11 22 -311 val_311 2008-04-08 11 22 -27 val_27 2008-04-08 11 34 -165 val_165 2008-04-08 11 44 -165 val_165 2008-04-08 11 44 -238 val_238 2008-04-08 12 0 -238 val_238 2008-04-08 12 0 -86 val_86 2008-04-08 12 12 -311 val_311 2008-04-08 12 22 -311 val_311 2008-04-08 12 22 -311 val_311 2008-04-08 12 22 -27 val_27 2008-04-08 12 34 -165 val_165 2008-04-08 12 44 -165 val_165 2008-04-08 12 44 -238 val_238 2008-04-09 11 0 -238 val_238 2008-04-09 11 0 -86 val_86 2008-04-09 11 12 -311 val_311 2008-04-09 11 22 -311 val_311 2008-04-09 11 22 -311 val_311 2008-04-09 11 22 -27 val_27 2008-04-09 11 34 -165 val_165 2008-04-09 11 44 -165 val_165 2008-04-09 11 44 -238 val_238 2008-04-09 12 0 -238 val_238 2008-04-09 12 0 -86 val_86 2008-04-09 12 12 -311 val_311 2008-04-09 12 22 -311 val_311 2008-04-09 12 22 -311 val_311 2008-04-09 12 22 -27 val_27 2008-04-09 12 34 -165 val_165 2008-04-09 12 44 -165 val_165 2008-04-09 12 44 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/ppr_pushdown-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/ppr_pushdown-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b b/sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-1-855b235f9c760ba9d6d0200bfd1ad08b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/ppr_pushdown-1-855b235f9c760ba9d6d0200bfd1ad08b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 b/sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 b/sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-10-2957fd9b211cee5f0372525a1de55c19 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/ppr_pushdown-10-2957fd9b211cee5f0372525a1de55c19 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 b/sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-11-b8d6f0ffc8294497c792b26958adee45 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown-11-b8d6f0ffc8294497c792b26958adee45 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 b/sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-12-ab1b9a0cdd9586c96d8856a9d632b89c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/ppr_pushdown-12-ab1b9a0cdd9586c96d8856a9d632b89c diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 b/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c316c67fd494d878fbbea107d283c3b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 rename to sql/hive/src/test/resources/golden/ppr_pushdown-13-2c316c67fd494d878fbbea107d283c3b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 b/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-14-53b4be82a1538844d03b200429efa02b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown-14-53b4be82a1538844d03b200429efa02b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c b/sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-15-71ab93d38ed2908069091c7c8cc0aba1 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/ppr_pushdown-15-71ab93d38ed2908069091c7c8cc0aba1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb b/sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae b/sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/ppr_pushdown-16-855cb54d28034fdb20a3615ee0918d63 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/ppr_pushdown-16-855cb54d28034fdb20a3615ee0918d63 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-17-d3ccf2722a8b7281fcee61b2544772c8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/ppr_pushdown-17-d3ccf2722a8b7281fcee61b2544772c8 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 b/sql/hive/src/test/resources/golden/ppr_pushdown-18-fcd10fbfc55a2c0aa843fe618f9613c6 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 rename to sql/hive/src/test/resources/golden/ppr_pushdown-18-fcd10fbfc55a2c0aa843fe618f9613c6 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 b/sql/hive/src/test/resources/golden/ppr_pushdown-19-ff36e3978571ac05f11e8322c024e4b6 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 rename to sql/hive/src/test/resources/golden/ppr_pushdown-19-ff36e3978571ac05f11e8322c024e4b6 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 b/sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-2-f689aaf15b08c433d1e93ce977a9b6d5 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/ppr_pushdown-2-f689aaf15b08c433d1e93ce977a9b6d5 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a b/sql/hive/src/test/resources/golden/ppr_pushdown-20-481005cf034ef3d7b998da32eb82aa9a similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a rename to sql/hive/src/test/resources/golden/ppr_pushdown-20-481005cf034ef3d7b998da32eb82aa9a diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 b/sql/hive/src/test/resources/golden/ppr_pushdown-21-9073825e8b9804331f780980d1f9fa92 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 rename to sql/hive/src/test/resources/golden/ppr_pushdown-21-9073825e8b9804331f780980d1f9fa92 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d b/sql/hive/src/test/resources/golden/ppr_pushdown-22-9cb28f0d3a434c9d1ab039192906ec9d similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d rename to sql/hive/src/test/resources/golden/ppr_pushdown-22-9cb28f0d3a434c9d1ab039192906ec9d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d b/sql/hive/src/test/resources/golden/ppr_pushdown-23-678f50025924fee7b59f66e2abdb472d similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d rename to sql/hive/src/test/resources/golden/ppr_pushdown-23-678f50025924fee7b59f66e2abdb472d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 b/sql/hive/src/test/resources/golden/ppr_pushdown-24-4d5bfa800ba434f464b07bf9b5d39f20 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 rename to sql/hive/src/test/resources/golden/ppr_pushdown-24-4d5bfa800ba434f464b07bf9b5d39f20 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 b/sql/hive/src/test/resources/golden/ppr_pushdown-25-22663f09ea1c1bc303756067e84df5a7 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 rename to sql/hive/src/test/resources/golden/ppr_pushdown-25-22663f09ea1c1bc303756067e84df5a7 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b b/sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 b/sql/hive/src/test/resources/golden/ppr_pushdown-27-8065c18e387fd8bffae19a80af8dc1d4 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 rename to sql/hive/src/test/resources/golden/ppr_pushdown-27-8065c18e387fd8bffae19a80af8dc1d4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 b/sql/hive/src/test/resources/golden/ppr_pushdown-28-b72de558c88ae91460989938000e0d27 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 rename to sql/hive/src/test/resources/golden/ppr_pushdown-28-b72de558c88ae91460989938000e0d27 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 b/sql/hive/src/test/resources/golden/ppr_pushdown-29-7217735d357770af4ffee730e4e9add4 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 rename to sql/hive/src/test/resources/golden/ppr_pushdown-29-7217735d357770af4ffee730e4e9add4 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-3-5eede4874e678021938683fc2f5dc900 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/ppr_pushdown-3-5eede4874e678021938683fc2f5dc900 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 b/sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea b/sql/hive/src/test/resources/golden/ppr_pushdown-30-630e2f7918b7727fc4ca057fa21e2eea similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea rename to sql/hive/src/test/resources/golden/ppr_pushdown-30-630e2f7918b7727fc4ca057fa21e2eea diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 b/sql/hive/src/test/resources/golden/ppr_pushdown-31-2e062414293b643ea4d7f6df92f939e4 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 rename to sql/hive/src/test/resources/golden/ppr_pushdown-31-2e062414293b643ea4d7f6df92f939e4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf b/sql/hive/src/test/resources/golden/ppr_pushdown-32-5eba4bf10315099129eae319d73636cf similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf rename to sql/hive/src/test/resources/golden/ppr_pushdown-32-5eba4bf10315099129eae319d73636cf diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b b/sql/hive/src/test/resources/golden/ppr_pushdown-33-35af0585a4f98bc222c786688cb6de6b similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b rename to sql/hive/src/test/resources/golden/ppr_pushdown-33-35af0585a4f98bc222c786688cb6de6b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 b/sql/hive/src/test/resources/golden/ppr_pushdown-34-a5594625510703427ab8bae5d0563f73 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 rename to sql/hive/src/test/resources/golden/ppr_pushdown-34-a5594625510703427ab8bae5d0563f73 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/ppr_pushdown-4-c5c542f8ee81cd0afd44e67fc7b4d306 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/ppr_pushdown-4-c5c542f8ee81cd0afd44e67fc7b4d306 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 b/sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de b/sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-5-f54bebec398f0fdfdbc0393123dba234 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/ppr_pushdown-5-f54bebec398f0fdfdbc0393123dba234 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d b/sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-6-78af12432bcbf99d4a0d41c25f964de similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/ppr_pushdown-6-78af12432bcbf99d4a0d41c25f964de diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown-7-58724fbe96a0b3deceef20c8cc5e318d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 rename to sql/hive/src/test/resources/golden/ppr_pushdown-7-58724fbe96a0b3deceef20c8cc5e318d diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown-8-14570f946e75924d7926c809485951d1 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/ppr_pushdown-8-14570f946e75924d7926c809485951d1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c b/sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 b/sql/hive/src/test/resources/golden/ppr_pushdown-9-3ba325662296fc455f07f1c835495e4c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 rename to sql/hive/src/test/resources/golden/ppr_pushdown-9-3ba325662296fc455f07f1c835495e4c diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 b/sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/ppr_pushdown2-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/ppr_pushdown2-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b b/sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 b/sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown2-1-855b235f9c760ba9d6d0200bfd1ad08b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-1-855b235f9c760ba9d6d0200bfd1ad08b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 new file mode 100644 index 0000000000000..f50a5fea8dd5d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 @@ -0,0 +1 @@ +1 1 2 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b b/sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown2-11-ab3e46183487096745d7d90e3020e94c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/ppr_pushdown2-11-ab3e46183487096745d7d90e3020e94c diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a b/sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-12-b10b9e14f5a7a7a92c0c68df6dbc656a similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-12-b10b9e14f5a7a7a92c0c68df6dbc656a diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown2-13-154f9859bd0822e287fbfdff12fd45ff similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/ppr_pushdown2-13-154f9859bd0822e287fbfdff12fd45ff diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 b/sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 deleted file mode 100644 index e3e04ee48543d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 +++ /dev/null @@ -1 +0,0 @@ -3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown2-14-cf4a30b5c8329d8d79ddf762f318fbb3 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-14-cf4a30b5c8329d8d79ddf762f318fbb3 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 deleted file mode 100644 index c458b0f57aba9..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 +++ /dev/null @@ -1 +0,0 @@ -2 1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 new file mode 100644 index 0000000000000..3def25c3c65af --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 @@ -0,0 +1 @@ +3 1 2 1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb deleted file mode 100644 index c458b0f57aba9..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb +++ /dev/null @@ -1 +0,0 @@ -2 1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 new file mode 100644 index 0000000000000..55c794b56ec9a --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 @@ -0,0 +1 @@ +2 1 1 2 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb new file mode 100644 index 0000000000000..55c794b56ec9a --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb @@ -0,0 +1 @@ +2 1 1 2 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe deleted file mode 100644 index 63511415ddf55..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe +++ /dev/null @@ -1,2 +0,0 @@ -3 1 2 1 -1 1 2 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe new file mode 100644 index 0000000000000..8d13286371dab --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe @@ -0,0 +1,2 @@ +3 1 2 1 +1 1 2 3 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 deleted file mode 100644 index c592b5d505b0e..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 +++ /dev/null @@ -1,3 +0,0 @@ -2 1 1 2 -3 1 2 1 -1 1 2 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 deleted file mode 100644 index e3e04ee48543d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 +++ /dev/null @@ -1 +0,0 @@ -3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 new file mode 100644 index 0000000000000..2bc7fedb12a50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 @@ -0,0 +1,3 @@ +2 1 1 2 +3 1 2 1 +1 1 2 3 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e b/sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown2-2-ffa167b63d612a4986d02f5c0623ea7b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/ppr_pushdown2-2-ffa167b63d612a4986d02f5c0623ea7b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 new file mode 100644 index 0000000000000..3def25c3c65af --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 @@ -0,0 +1 @@ +3 1 2 1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 deleted file mode 100644 index e3e04ee48543d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 +++ /dev/null @@ -1 +0,0 @@ -3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 b/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 new file mode 100644 index 0000000000000..3def25c3c65af --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 @@ -0,0 +1 @@ +3 1 2 1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c b/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c deleted file mode 100644 index 679926f7d738c..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c +++ /dev/null @@ -1 +0,0 @@ -2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-3-c7753746c190414723d66a8f876499c7 similarity index 100% rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-3-c7753746c190414723d66a8f876499c7 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c new file mode 100644 index 0000000000000..bfde072a7963c --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c @@ -0,0 +1 @@ +2 2 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 deleted file mode 100644 index fcc7be2cb12aa..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 +++ /dev/null @@ -1 +0,0 @@ -22 22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 new file mode 100644 index 0000000000000..38212d1943095 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 @@ -0,0 +1 @@ +22 22 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 b/sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba b/sql/hive/src/test/resources/golden/ppr_pushdown2-6-b5a2518af801f95fe52a75dfc1d3e867 similarity index 100% rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba rename to sql/hive/src/test/resources/golden/ppr_pushdown2-6-b5a2518af801f95fe52a75dfc1d3e867 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 b/sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 b/sql/hive/src/test/resources/golden/ppr_pushdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b similarity index 100% rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-8-4507a3f200b3ce384191c91acd324dc7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-8-4507a3f200b3ce384191c91acd324dc7 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 b/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 deleted file mode 100644 index 3ebc6d7fa2375..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 +++ /dev/null @@ -1 +0,0 @@ -2 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 deleted file mode 100644 index a4544ab84afa3..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 +++ /dev/null @@ -1 +0,0 @@ -1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 new file mode 100644 index 0000000000000..699fa0cd95c4f --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 @@ -0,0 +1 @@ +2 2 1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a b/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 b/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 index e22f840876d2e..546fd0b0e1051 100644 --- a/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 @@ -37,4 +37,4 @@ 5 val_5 2008-04-09 12 2 val_2 2008-04-09 12 5 val_5 2008-04-09 12 -9 val_9 2008-04-09 12 \ No newline at end of file +9 val_9 2008-04-09 12 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb b/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb index 355ed1617e200..2857cdf0aba86 100644 --- a/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb @@ -1997,4 +1997,4 @@ 403 val_403 2008-04-09 12 400 val_400 2008-04-09 12 200 val_200 2008-04-09 12 -97 val_97 2008-04-09 12 \ No newline at end of file +97 val_97 2008-04-09 12 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f b/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f index 57005044dde38..7f3ca6e01ea06 100644 --- a/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f @@ -1997,4 +1997,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 b/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a b/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a deleted file mode 100644 index 943e1be13b615..0000000000000 --- a/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a +++ /dev/null @@ -1,10 +0,0 @@ -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 2 val_2 diff --git a/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 b/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 deleted file mode 100644 index 951e74db0fe23..0000000000000 --- a/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 +++ /dev/null @@ -1,10 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 diff --git a/sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 b/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 +++ b/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f b/sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 b/sql/hive/src/test/resources/golden/progress_1-2-b6c8c8fc9df98af4dead5efabf5f162c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 rename to sql/hive/src/test/resources/golden/progress_1-2-b6c8c8fc9df98af4dead5efabf5f162c diff --git a/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 b/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 index 0b3e0a69a8c79..e9c02dad1826a 100644 --- a/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 +++ b/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 @@ -1 +1 @@ -5000 \ No newline at end of file +5000 diff --git a/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 b/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 index eb63636d0bfd4..41b92dbf0214a 100644 --- a/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 +++ b/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1388801768, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388801769, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801769}) \ No newline at end of file +Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1413890256, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890256, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d index 964ede006ad21..27254dd97006d 100644 --- a/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d +++ b/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d index 68b03670f5f80..4d09f8c357ea7 100644 --- a/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d +++ b/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=OFFLINE,NO_DROP, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE,NO_DROP, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d index ec18ce7411535..dbc128d2a1d8c 100644 --- a/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d +++ b/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d index a6202ac5fc0eb..7774c774cc8a1 100644 --- a/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d +++ b/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801771, transient_lastDdlTime=1388801771}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 b/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 index 0a6cebbbd0b82..567b9b3a5d228 100644 --- a/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 +++ b/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1388801768, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388801783, transient_lastDdlTime=1388801783}) \ No newline at end of file +Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1413890256, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890263, transient_lastDdlTime=1413890263, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 b/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 index a1ff1f8341f30..be56722166fe4 100644 --- a/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 +++ b/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 @@ -1,3 +1,3 @@ -col string None +col string -Detailed Table Information Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1388801746, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388801756, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801756}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1413890242, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890249, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890249, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 b/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 index 9098c9419556b..a3c3c67860fdf 100644 --- a/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 +++ b/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 @@ -1,3 +1,3 @@ -col string None +col string -Detailed Table Information Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1388801746, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388801756, transient_lastDdlTime=1388801756}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1413890242, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890249, transient_lastDdlTime=1413890249, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a b/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a index 3a2f20c637883..e3c6c66098c10 100644 --- a/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a +++ b/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a @@ -20,4 +20,4 @@ 17 val_17 2000-04-09 18 val_18 2000-04-09 18 val_18 2000-04-09 -19 val_19 2000-04-09 \ No newline at end of file +19 val_19 2000-04-09 diff --git a/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 b/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 index 2f3fe0189d2c6..10db2ad303244 100644 --- a/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 +++ b/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 @@ -1 +1 @@ -aa; \ No newline at end of file +aa; diff --git a/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb b/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb index b5b5773c405b4..e0b3f1b09bd18 100644 --- a/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb +++ b/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb @@ -1 +1 @@ -bb \ No newline at end of file +bb diff --git a/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 b/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 index 2652f5f42c003..46c1d6125b7b4 100644 --- a/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 +++ b/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 @@ -1 +1 @@ -cc \ No newline at end of file +cc diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/quote2-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/quote2-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 b/sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 b/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 deleted file mode 100644 index ed1f53a6588d0..0000000000000 --- a/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 +++ /dev/null @@ -1 +0,0 @@ -abc abc abc' abc" abc\ abc\ abc\' abc\" abc\\ abc\\ abc\\' abc\\" abc\\\ abc\\\ abc""""\ abc''''\ awk '{print NR"\t"$0}' tab tab tab tab \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 b/sql/hive/src/test/resources/golden/quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 rename to sql/hive/src/test/resources/golden/quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118 diff --git a/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 b/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 new file mode 100644 index 0000000000000..4c8564d085999 --- /dev/null +++ b/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 @@ -0,0 +1 @@ +abc abc abc' abc" abc\ abc\ abc\' abc\" abc\\ abc\\ abc\\' abc\\" abc\\\ abc\\\ abc""""\ abc''''\ awk '{print NR"\t"$0}' tab tab tab tab diff --git a/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e b/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e +++ b/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d b/sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 b/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 deleted file mode 100644 index 73b8edab36833..0000000000000 --- a/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 +++ /dev/null @@ -1,56 +0,0 @@ -409 val_409 -429 val_429 -209 val_209 -153 val_153 -203 val_203 -170 val_170 -489 val_489 -378 val_378 -221 val_221 -498 val_498 -469 val_469 -176 val_176 -176 val_176 -384 val_384 -217 val_217 -431 val_431 -51 val_51 -288 val_288 -457 val_457 -197 val_197 -77 val_77 -138 val_138 -277 val_277 -224 val_224 -309 val_309 -389 val_389 -331 val_331 -317 val_317 -336 val_336 -42 val_42 -458 val_458 -78 val_78 -453 val_453 -74 val_74 -103 val_103 -467 val_467 -202 val_202 -469 val_469 -44 val_44 -454 val_454 -70 val_70 -491 val_491 -199 val_199 -169 val_169 -310 val_310 -233 val_233 -133 val_133 -26 val_26 -134 val_134 -18 val_18 -298 val_298 -348 val_348 -469 val_469 -37 val_37 -152 val_152 -400 val_400 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 deleted file mode 100644 index e8738e9c837ee..0000000000000 --- a/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 +++ /dev/null @@ -1,116 +0,0 @@ -103 val_103 2008-04-08 11 -118 val_118 2008-04-08 12 -119 val_119 2008-04-08 12 -119 val_119 2008-04-08 12 -126 val_126 2008-04-08 12 -131 val_131 2008-04-08 12 -133 val_133 2008-04-08 11 -134 val_134 2008-04-08 11 -138 val_138 2008-04-08 11 -143 val_143 2008-04-08 12 -152 val_152 2008-04-08 11 -153 val_153 2008-04-08 11 -162 val_162 2008-04-08 12 -169 val_169 2008-04-08 11 -170 val_170 2008-04-08 11 -175 val_175 2008-04-08 12 -176 val_176 2008-04-08 11 -176 val_176 2008-04-08 11 -18 val_18 2008-04-08 11 -18 val_18 2008-04-08 12 -191 val_191 2008-04-08 12 -197 val_197 2008-04-08 11 -199 val_199 2008-04-08 11 -200 val_200 2008-04-08 12 -201 val_201 2008-04-08 12 -202 val_202 2008-04-08 11 -203 val_203 2008-04-08 11 -209 val_209 2008-04-08 11 -214 val_214 2008-04-08 12 -217 val_217 2008-04-08 11 -218 val_218 2008-04-08 12 -221 val_221 2008-04-08 11 -223 val_223 2008-04-08 12 -224 val_224 2008-04-08 11 -229 val_229 2008-04-08 12 -230 val_230 2008-04-08 12 -233 val_233 2008-04-08 11 -233 val_233 2008-04-08 12 -237 val_237 2008-04-08 12 -238 val_238 2008-04-08 12 -256 val_256 2008-04-08 12 -26 val_26 2008-04-08 11 -265 val_265 2008-04-08 12 -273 val_273 2008-04-08 12 -277 val_277 2008-04-08 11 -277 val_277 2008-04-08 12 -280 val_280 2008-04-08 12 -286 val_286 2008-04-08 12 -288 val_288 2008-04-08 11 -298 val_298 2008-04-08 11 -309 val_309 2008-04-08 11 -309 val_309 2008-04-08 12 -310 val_310 2008-04-08 11 -317 val_317 2008-04-08 11 -322 val_322 2008-04-08 12 -323 val_323 2008-04-08 12 -325 val_325 2008-04-08 12 -331 val_331 2008-04-08 11 -332 val_332 2008-04-08 12 -336 val_336 2008-04-08 11 -336 val_336 2008-04-08 12 -339 val_339 2008-04-08 12 -341 val_341 2008-04-08 12 -342 val_342 2008-04-08 12 -348 val_348 2008-04-08 11 -348 val_348 2008-04-08 12 -35 val_35 2008-04-08 12 -364 val_364 2008-04-08 12 -37 val_37 2008-04-08 11 -378 val_378 2008-04-08 11 -384 val_384 2008-04-08 11 -389 val_389 2008-04-08 11 -400 val_400 2008-04-08 11 -403 val_403 2008-04-08 12 -407 val_407 2008-04-08 12 -409 val_409 2008-04-08 11 -417 val_417 2008-04-08 12 -42 val_42 2008-04-08 11 -424 val_424 2008-04-08 12 -429 val_429 2008-04-08 11 -429 val_429 2008-04-08 12 -430 val_430 2008-04-08 12 -431 val_431 2008-04-08 11 -432 val_432 2008-04-08 12 -44 val_44 2008-04-08 11 -453 val_453 2008-04-08 11 -454 val_454 2008-04-08 11 -457 val_457 2008-04-08 11 -457 val_457 2008-04-08 12 -458 val_458 2008-04-08 11 -466 val_466 2008-04-08 12 -467 val_467 2008-04-08 11 -469 val_469 2008-04-08 11 -469 val_469 2008-04-08 11 -469 val_469 2008-04-08 11 -47 val_47 2008-04-08 12 -470 val_470 2008-04-08 12 -489 val_489 2008-04-08 11 -491 val_491 2008-04-08 11 -496 val_496 2008-04-08 12 -498 val_498 2008-04-08 11 -498 val_498 2008-04-08 12 -51 val_51 2008-04-08 11 -58 val_58 2008-04-08 12 -70 val_70 2008-04-08 11 -72 val_72 2008-04-08 12 -74 val_74 2008-04-08 11 -77 val_77 2008-04-08 11 -77 val_77 2008-04-08 12 -78 val_78 2008-04-08 11 -82 val_82 2008-04-08 12 -87 val_87 2008-04-08 12 -90 val_90 2008-04-08 12 -97 val_97 2008-04-08 12 -97 val_97 2008-04-08 12 -98 val_98 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 b/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a b/sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b b/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b index d1b3011bffd91..2d2bcebee4a8d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b +++ b/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b @@ -1,2 +1,2 @@ -key string None -value string None +key string +value string diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 b/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 b/sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c b/sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 b/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 deleted file mode 100644 index b141fed81f206..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 +++ /dev/null @@ -1 +0,0 @@ -53278638794 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 b/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 deleted file mode 100644 index b141fed81f206..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 +++ /dev/null @@ -1 +0,0 @@ -53278638794 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 b/sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e b/sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee b/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 b/sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e b/sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 b/sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 b/sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f b/sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 b/sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 b/sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 b/sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 b/sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a b/sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 b/sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 b/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 deleted file mode 100644 index 8d25d618795b9..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 +++ /dev/null @@ -1 +0,0 @@ --4208881187 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 b/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 deleted file mode 100644 index 8d25d618795b9..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 +++ /dev/null @@ -1 +0,0 @@ --4208881187 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 b/sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 b/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 deleted file mode 100644 index 11ff946b46f0f..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 +++ /dev/null @@ -1 +0,0 @@ -14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f b/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f deleted file mode 100644 index 11ff946b46f0f..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f +++ /dev/null @@ -1 +0,0 @@ -14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 b/sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b b/sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 b/sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 b/sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 b/sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 b/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 deleted file mode 100644 index 11ff946b46f0f..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 +++ /dev/null @@ -1 +0,0 @@ -14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f b/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f deleted file mode 100644 index 11ff946b46f0f..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f +++ /dev/null @@ -1 +0,0 @@ -14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 b/sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 b/sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 b/sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 b/sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee b/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee index b0135533064c9..2918f17e964c0 100644 --- a/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee +++ b/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee @@ -22,4 +22,4 @@ NULL val_193 406 val_406 NULL NULL -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 b/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 index 06b2b4d7e6d47..b52cff5c472e4 100644 --- a/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 +++ b/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 @@ -6,4 +6,4 @@ NULL NULL 24 val_24 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b deleted file mode 100644 index 60878ffb77064..0000000000000 --- a/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b +++ /dev/null @@ -1 +0,0 @@ -238 val_238 diff --git a/sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 b/sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 b/sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db b/sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 b/sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 b/sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 b/sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 b/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa b/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 b/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d b/sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b index 726e0947d8302..017e14d2ebed4 100644 --- a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a int None -b int None -c int None \ No newline at end of file +a int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b index 500201be8d922..a92663b0674bf 100644 --- a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None +b int a1 int test comment1 -c int None \ No newline at end of file +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b index 71af16a5fa4d7..899341a881857 100644 --- a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ a2 int test comment2 -b int None -c int None \ No newline at end of file +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b index f5b2e72aeccab..26b38dcc6d855 100644 --- a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None +b int a int test comment2 -c int None \ No newline at end of file +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c index ecafeaea5f61a..85c1918f46567 100644 --- a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c @@ -1,2 +1,2 @@ src -srcpart \ No newline at end of file +srcpart diff --git a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b index 726e0947d8302..017e14d2ebed4 100644 --- a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a int None -b int None -c int None \ No newline at end of file +a int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b index 17127eaec9755..2fbb615dd5994 100644 --- a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a string None -b int None -c int None \ No newline at end of file +a string +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b index bfcefe4d18046..173fbad7b1eb3 100644 --- a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a1 int None -b int None -c int None \ No newline at end of file +a1 int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b index c436c39a16b8a..bad9feb96a886 100644 --- a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a2 int None -b int None -c int None \ No newline at end of file +a2 int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b index bb1507e7488f5..4f23db53afff2 100644 --- a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None -a int None -c int None \ No newline at end of file +b int +a int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b index 17127eaec9755..2fbb615dd5994 100644 --- a/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a string None -b int None -c int None \ No newline at end of file +a string +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b index 500201be8d922..a92663b0674bf 100644 --- a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None +b int a1 int test comment1 -c int None \ No newline at end of file +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b index 71af16a5fa4d7..899341a881857 100644 --- a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ a2 int test comment2 -b int None -c int None \ No newline at end of file +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b index f5b2e72aeccab..26b38dcc6d855 100644 --- a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None +b int a int test comment2 -c int None \ No newline at end of file +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b index bfcefe4d18046..173fbad7b1eb3 100644 --- a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a1 int None -b int None -c int None \ No newline at end of file +a1 int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b index c436c39a16b8a..bad9feb96a886 100644 --- a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a2 int None -b int None -c int None \ No newline at end of file +a2 int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b index bb1507e7488f5..4f23db53afff2 100644 --- a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None -a int None -c int None \ No newline at end of file +b int +a int +c int diff --git a/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 b/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 b/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 index 3a57720041fb3..d091388cd5e19 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 +++ b/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 b/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 index 19492fd335bcb..a86e9c5af723e 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 +++ b/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f b/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f index 3a57720041fb3..d091388cd5e19 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f +++ b/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e b/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e index 19492fd335bcb..a86e9c5af723e 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e +++ b/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 b/sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 b/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 deleted file mode 100644 index 0bc999a3e6aa2..0000000000000 --- a/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef b/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 b/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 deleted file mode 100644 index 0bc999a3e6aa2..0000000000000 --- a/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 b/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 b/sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 b/sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 b/sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f b/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f deleted file mode 100644 index 9cf5170e82d7d..0000000000000 --- a/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f +++ /dev/null @@ -1 +0,0 @@ -238 val_238 238 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 b/sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 b/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 b/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 new file mode 100644 index 0000000000000..a81afd7b04319 --- /dev/null +++ b/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 @@ -0,0 +1,11 @@ +1712634731 record_0 [0,0,0] ["0","0","0"] [{"myint":0,"mystring":"0","underscore_int":0}] {"key_0":"value_0"} +465985200 record_1 [1,2,3] ["10","100","1000"] [{"myint":1,"mystring":"1","underscore_int":1}] {"key_1":"value_1"} +-751827638 record_2 [2,4,6] ["20","200","2000"] [{"myint":4,"mystring":"8","underscore_int":2}] {"key_2":"value_2"} +477111222 record_3 [3,6,9] ["30","300","3000"] [{"myint":9,"mystring":"27","underscore_int":3}] {"key_3":"value_3"} +-734328909 record_4 [4,8,12] ["40","400","4000"] [{"myint":16,"mystring":"64","underscore_int":4}] {"key_4":"value_4"} +-1952710710 record_5 [5,10,15] ["50","500","5000"] [{"myint":25,"mystring":"125","underscore_int":5}] {"key_5":"value_5"} +1244525190 record_6 [6,12,18] ["60","600","6000"] [{"myint":36,"mystring":"216","underscore_int":6}] {"key_6":"value_6"} +-1461153973 record_7 [7,14,21] ["70","700","7000"] [{"myint":49,"mystring":"343","underscore_int":7}] {"key_7":"value_7"} +1638581578 record_8 [8,16,24] ["80","800","8000"] [{"myint":64,"mystring":"512","underscore_int":8}] {"key_8":"value_8"} +336964413 record_9 [9,18,27] ["90","900","9000"] [{"myint":81,"mystring":"729","underscore_int":9}] {"key_9":"value_9"} +0 NULL NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 b/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 index 5f4de85940513..016f64cc26f2a 100644 --- a/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 +++ b/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 @@ -1 +1 @@ -0 val_0 \ No newline at end of file +0 val_0 diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e index f35d9602e1a7b..f9aaa4d565f4a 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e +++ b/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e @@ -1 +1 @@ -498 \ No newline at end of file +498 diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 b/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 +++ b/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e index f35d9602e1a7b..f9aaa4d565f4a 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e +++ b/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e @@ -1 +1 @@ -498 \ No newline at end of file +498 diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde b/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde +++ b/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e index f35d9602e1a7b..f9aaa4d565f4a 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e +++ b/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e @@ -1 +1 @@ -498 \ No newline at end of file +498 diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f b/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f +++ b/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e index d00ee7786a57c..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e +++ b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e @@ -1,22 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATETABLE (TOK_TABNAME serde_regex) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL host TOK_STRING) (TOK_TABCOL identity TOK_STRING) (TOK_TABCOL user TOK_STRING) (TOK_TABCOL time TOK_STRING) (TOK_TABCOL request TOK_STRING) (TOK_TABCOL status TOK_STRING) (TOK_TABCOL size TOK_INT) (TOK_TABCOL referer TOK_STRING) (TOK_TABCOL agent TOK_STRING)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?"))))) TOK_TBLTEXTFILE) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Create Table Operator: - Create Table - columns: host string, identity string, user string, time string, request string, status string, size int, referer string, agent string - if not exists: false - input format: org.apache.hadoop.mapred.TextInputFormat - # buckets: -1 - output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat - serde name: org.apache.hadoop.hive.serde2.RegexSerDe - serde properties: - input.regex ([^ ]*) ([^ ]*) ([^ ]*) (-|\[[^\]]*\]) ([^ "]*|"[^"]*") (-|[0-9]*) (-|[0-9]*)(?: ([^ "]*|"[^"]*") ([^ "]*|"[^"]*"))? - name: serde_regex - isExternal: false - - diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 index 3e290231c27e2..93cdc5c85645c 100644 --- a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 +++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 @@ -1,5 +1,4 @@ NULL 0 -NULL 0 -1234567890.123456789 -1234567890 -4400 4400 -1255.49 -1255 @@ -11,6 +10,7 @@ NULL 0 -0.3 0 0 0 0 0 +0 0 0.01 0 0.02 0 0.1 0 @@ -18,7 +18,7 @@ NULL 0 0.3 0 0.33 0 0.333 0 -0.9999999999999999999999999 1 +1 1 1 1 1 1 1.12 1 diff --git a/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 b/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 b/sql/hive/src/test/resources/golden/serde_regex-2-e84d30fcc6cf11e82b54ea63e7d1d611 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 rename to sql/hive/src/test/resources/golden/serde_regex-2-e84d30fcc6cf11e82b54ea63e7d1d611 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 b/sql/hive/src/test/resources/golden/serde_regex-3-3ee9e78ff563d6b48741a41885f92c81 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 rename to sql/hive/src/test/resources/golden/serde_regex-3-3ee9e78ff563d6b48741a41885f92c81 diff --git a/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc b/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 deleted file mode 100644 index da61769c6599d..0000000000000 --- a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 +++ /dev/null @@ -1,22 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATETABLE (TOK_TABNAME serde_regex1) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL key TOK_DECIMAL) (TOK_TABCOL value TOK_INT)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*)"))))) TOK_TBLTEXTFILE) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Create Table Operator: - Create Table - columns: key decimal, value int - if not exists: false - input format: org.apache.hadoop.mapred.TextInputFormat - # buckets: -1 - output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat - serde name: org.apache.hadoop.hive.serde2.RegexSerDe - serde properties: - input.regex ([^ ]*) ([^ ]*) - name: serde_regex1 - isExternal: false - - diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 b/sql/hive/src/test/resources/golden/serde_regex-7-bf456bcf6be7334488424dfeadf27d75 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 rename to sql/hive/src/test/resources/golden/serde_regex-7-bf456bcf6be7334488424dfeadf27d75 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 b/sql/hive/src/test/resources/golden/serde_regex-8-a4cf34af32b83e40e5c8b6d083938b54 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 rename to sql/hive/src/test/resources/golden/serde_regex-8-a4cf34af32b83e40e5c8b6d083938b54 diff --git a/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 b/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b b/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 b/sql/hive/src/test/resources/golden/serde_regex-9-f1175f3322abec6f258dd49a5905bce0 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 rename to sql/hive/src/test/resources/golden/serde_regex-9-f1175f3322abec6f258dd49a5905bce0 diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 b/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 index d1168556e09d4..b1663e9a8c00d 100644 --- a/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 +++ b/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 @@ -1,11 +1,11 @@ myint int from deserializer mystring string from deserializer underscore_int int from deserializer -b string None +b string # Partition Information # col_name data_type comment -b string None +b string -Detailed Table Information Table(tableName:int_string, dbName:default, owner:marmbrus, createTime:1389733035, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/int_string, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:b, type:string, comment:null)], parameters:{transient_lastDdlTime=1389733035}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:int_string, dbName:default, owner:marmbrus, createTime:1413891326, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/int_string, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:b, type:string, comment:null)], parameters:{transient_lastDdlTime=1413891326}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa b/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa index 0b8f428b24193..f5ec7a9aca8f3 100644 --- a/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa +++ b/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa @@ -1,11 +1,11 @@ myint int from deserializer mystring string from deserializer underscore_int int from deserializer -b string None +b string # Partition Information # col_name data_type comment -b string None +b string -Detailed Partition Information Partition(values:[part1], dbName:default, tableName:int_string, createTime:1389733036, lastAccessTime:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/int_string/b=part1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389733036}) \ No newline at end of file +Detailed Partition Information Partition(values:[part1], dbName:default, tableName:int_string, createTime:1413891326, lastAccessTime:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/int_string/b=part1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413891326}) diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d b/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d +++ b/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 b/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 +++ b/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 b/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 +++ b/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 b/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 +++ b/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd b/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd +++ b/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b b/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b +++ b/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d b/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d +++ b/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 b/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 +++ b/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 index 3c1fc128bedce..69c7709aa90cb 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key smallint, - value float) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) CLUSTERED BY ( key) SORTED BY ( @@ -13,6 +13,6 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132100') + 'transient_lastDdlTime'='1413891329') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 index 2ece813dd7d56..501bb6ab32f25 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE TABLE tmp_showcrt1( - key smallint, - value float) +CREATE TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) COMMENT 'temporary table' CLUSTERED BY ( key) @@ -14,9 +14,14 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( + 'numFiles'='0', 'EXTERNAL'='FALSE', - 'last_modified_by'='tianyi', - 'last_modified_time'='1407132100', - 'transient_lastDdlTime'='1407132100') + 'last_modified_by'='marmbrus', + 'last_modified_time'='1413891329', + 'transient_lastDdlTime'='1413891329', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 index 2af657bd29506..6e353675b5ed8 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key smallint, - value float) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) COMMENT 'changed comment' CLUSTERED BY ( key) @@ -14,8 +14,13 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'last_modified_by'='tianyi', - 'last_modified_time'='1407132100', - 'transient_lastDdlTime'='1407132100') + 'numFiles'='0', + 'last_modified_by'='marmbrus', + 'last_modified_time'='1413891329', + 'transient_lastDdlTime'='1413891329', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 index f793ffb7a0bfd..6e353675b5ed8 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key smallint, - value float) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) COMMENT 'changed comment' CLUSTERED BY ( key) @@ -14,8 +14,13 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'last_modified_by'='tianyi', - 'last_modified_time'='1407132101', - 'transient_lastDdlTime'='1407132101') + 'numFiles'='0', + 'last_modified_by'='marmbrus', + 'last_modified_time'='1413891329', + 'transient_lastDdlTime'='1413891329', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 index c65aff26a7fc1..da849512f4d8f 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key smallint, - value float) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) COMMENT 'changed comment' CLUSTERED BY ( key) @@ -14,8 +14,13 @@ STORED BY WITH SERDEPROPERTIES ( 'serialization.format'='1') LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'last_modified_by'='tianyi', - 'last_modified_time'='1407132101', - 'transient_lastDdlTime'='1407132101') + 'numFiles'='0', + 'last_modified_by'='marmbrus', + 'last_modified_time'='1413891329', + 'transient_lastDdlTime'='1413891329', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 index b5a18368ed85e..90f8415a1c6be 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 +++ b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 @@ -1,6 +1,6 @@ -CREATE TABLE tmp_feng.tmp_showcrt( - key string, - value int) +CREATE TABLE `tmp_feng.tmp_showcrt`( + `key` string, + `value` int) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' STORED AS INPUTFORMAT @@ -8,6 +8,6 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_feng.db/tmp_showcrt' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_feng.db/tmp_showcrt' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132107') + 'transient_lastDdlTime'='1413891330') diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 b/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 index d36ad25dc8273..4ee22e5230316 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 @@ -1,7 +1,7 @@ -CREATE TABLE tmp_showcrt1( - key int, - value string, - newvalue bigint) +CREATE TABLE `tmp_showcrt1`( + `key` int, + `value` string, + `newvalue` bigint) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' COLLECTION ITEMS TERMINATED BY '|' @@ -12,6 +12,6 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/testTempFiles4427612185729633290spark.hive.tmp/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132730') + 'transient_lastDdlTime'='1413891331') diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 index 9e572c0d7df6a..2a1acca6efb8d 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 @@ -1,9 +1,9 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key string, - newvalue boolean COMMENT 'a new value') +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` string, + `newvalue` boolean COMMENT 'a new value') COMMENT 'temporary table' PARTITIONED BY ( - value bigint COMMENT 'some value') + `value` bigint COMMENT 'some value') ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' STORED AS INPUTFORMAT @@ -11,6 +11,6 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132112') + 'transient_lastDdlTime'='1413891331') diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 index 69a38e1a7b20a..6fda2570b53f1 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 @@ -1,7 +1,7 @@ -CREATE TABLE tmp_showcrt1( - key int, - value string, - newvalue bigint) +CREATE TABLE `tmp_showcrt1`( + `key` int, + `value` string, + `newvalue` bigint) COMMENT 'temporary table' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' @@ -10,6 +10,11 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132115') + 'numFiles'='0', + 'transient_lastDdlTime'='1413891332', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 index b4e693dc622fb..cbbbb7b3ce3c7 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key string, - value boolean) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` string, + `value` boolean) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' STORED BY @@ -9,6 +9,6 @@ WITH SERDEPROPERTIES ( 'serialization.format'='$', 'field.delim'=',') LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132115') + 'transient_lastDdlTime'='1413891332') diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 index be3fb3ce30960..a721f07bb90b7 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 +++ b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 @@ -1 +1 @@ -CREATE VIEW tmp_copy_src AS SELECT `src`.`key`, `src`.`value` FROM `default`.`src` +CREATE VIEW `tmp_copy_src` AS SELECT `src`.`key`, `src`.`value` FROM `default`.`src` diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 index fd33cfcc9ab20..442b05b4cf4ee 100644 --- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 @@ -1 +1 @@ -concat \ No newline at end of file +concat diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d b/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d index fd33cfcc9ab20..442b05b4cf4ee 100644 --- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d @@ -1 +1 @@ -concat \ No newline at end of file +concat diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 index a3998eeeee623..b576089faa484 100644 --- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 @@ -1 +1 @@ -concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN \ No newline at end of file +concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 index a3998eeeee623..b576089faa484 100644 --- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 @@ -1 +1 @@ -concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN \ No newline at end of file +concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN diff --git a/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 b/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 index 9d40ffaef5862..3049cd6243ad8 100644 --- a/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 +++ b/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 @@ -32,6 +32,7 @@ case ceil ceiling coalesce +collect_list collect_set compute_stats concat @@ -45,6 +46,7 @@ covar_pop covar_samp create_union cume_dist +current_database date_add date_sub datediff @@ -123,6 +125,7 @@ percentile percentile_approx pi pmod +posexplode positive pow power @@ -189,4 +192,4 @@ xpath_short xpath_string year | -~ \ No newline at end of file +~ diff --git a/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 index c62b965cb1559..175795534fff5 100644 --- a/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 +++ b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 @@ -2,6 +2,7 @@ case ceil ceiling coalesce +collect_list collect_set compute_stats concat @@ -14,4 +15,5 @@ count covar_pop covar_samp create_union -cume_dist \ No newline at end of file +cume_dist +current_database diff --git a/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c index a56b5a3766c5c..3c25d656bda1c 100644 --- a/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c +++ b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c @@ -1,6 +1,7 @@ assert_true case coalesce +current_database decode e encode @@ -19,6 +20,7 @@ negative ntile parse_url_tuple percentile +posexplode positive regexp_replace reverse @@ -29,4 +31,4 @@ to_date translate ucase variance -xpath_double \ No newline at end of file +xpath_double diff --git a/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 b/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 index 86605075c3d25..b5a372a5ff50a 100644 --- a/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 +++ b/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 @@ -1,3 +1,3 @@ log log10 -log2 \ No newline at end of file +log2 diff --git a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 index 312f6cdbf68c7..cd2e58d04a4ef 100644 --- a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 +++ b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 @@ -1,4 +1,4 @@ date_add date_sub datediff -to_date \ No newline at end of file +to_date diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b deleted file mode 100644 index e772f4a83fecd..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_full) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 deleted file mode 100644 index d68fbfc9c1e0f..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 +++ /dev/null @@ -1,4 +0,0 @@ -idx_1 show_idx_full key default__show_idx_full_idx_1__ compact -idx_2 show_idx_full value1 default__show_idx_full_idx_2__ compact -idx_comment show_idx_full value2 default__show_idx_full_idx_comment__ compact index comment -idx_compound show_idx_full key, value1 default__show_idx_full_idx_compound__ compact \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e deleted file mode 100644 index 94ee57be9dcaf..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_empty) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec b/sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 b/sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec b/sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 b/sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 b/sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 b/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 deleted file mode 100644 index 7e68a8acb1a87..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_t1) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 b/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 deleted file mode 100644 index 36d22451eba3e..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 +++ /dev/null @@ -1 +0,0 @@ -idx_t1 show_idx_t1 key default__show_idx_t1_idx_t1__ compact \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 b/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 deleted file mode 100644 index 7e68a8acb1a87..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_t1) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 b/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 deleted file mode 100644 index 36d22451eba3e..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 +++ /dev/null @@ -1 +0,0 @@ -idx_t1 show_idx_t1 key default__show_idx_t1_idx_t1__ compact \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e b/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e deleted file mode 100644 index 4dddeee9a233f..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_t1 FORMATTED) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 b/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 deleted file mode 100644 index 76e0434294b4f..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 +++ /dev/null @@ -1,4 +0,0 @@ -idx_name tab_name col_names idx_tab_name idx_type comment - - -idx_t1 show_idx_t1 key default__show_idx_t1_idx_t1__ compact \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a b/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a index e9c723bbd136e..8c43153cf66f9 100644 --- a/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a +++ b/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a @@ -1,4 +1,4 @@ ds=2008-04-08/hr=11 ds=2008-04-08/hr=12 ds=2008-04-09/hr=11 -ds=2008-04-09/hr=12 \ No newline at end of file +ds=2008-04-09/hr=12 diff --git a/sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/show_partitions-1-e69b801a3c6c5f6692050bcdb0e31db9 similarity index 100% rename from sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 rename to sql/hive/src/test/resources/golden/show_partitions-1-e69b801a3c6c5f6692050bcdb0e31db9 diff --git a/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 b/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 deleted file mode 100644 index 19b4a62499762..0000000000000 --- a/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 +++ /dev/null @@ -1,2 +0,0 @@ -ds=2008-04-08/hr=11 -ds=2008-04-09/hr=11 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca b/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca deleted file mode 100644 index f3614273fa8fa..0000000000000 --- a/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca +++ /dev/null @@ -1,2 +0,0 @@ -ds=2008-04-08/hr=11 -ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 b/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 new file mode 100644 index 0000000000000..8b3fd053b6fb6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 @@ -0,0 +1,2 @@ +ds=2008-04-08/hr=11 +ds=2008-04-09/hr=11 diff --git a/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 b/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 deleted file mode 100644 index 0cdd3e8594c59..0000000000000 --- a/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 +++ /dev/null @@ -1 +0,0 @@ -ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 b/sql/hive/src/test/resources/golden/show_partitions-3-a1bde7c2c040b4d45ddceac9983c2ca similarity index 100% rename from sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 rename to sql/hive/src/test/resources/golden/show_partitions-3-a1bde7c2c040b4d45ddceac9983c2ca diff --git a/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 b/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 new file mode 100644 index 0000000000000..dbd11ad78405b --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 @@ -0,0 +1 @@ +ds=2008-04-08/hr=12 diff --git a/sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 b/sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce b/sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 b/sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 b/sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c b/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c deleted file mode 100644 index 60f7943eda4a9..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c +++ /dev/null @@ -1,3 +0,0 @@ -bar -baz -foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 b/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 deleted file mode 100644 index 4ffc580e2b8f3..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 +++ /dev/null @@ -1,4 +0,0 @@ -shtb_test1 -shtb_test2 -src -srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c b/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c deleted file mode 100644 index 60f7943eda4a9..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c +++ /dev/null @@ -1,3 +0,0 @@ -bar -baz -foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe b/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe deleted file mode 100644 index 4ffc580e2b8f3..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe +++ /dev/null @@ -1,4 +0,0 @@ -shtb_test1 -shtb_test2 -src -srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 b/sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab b/sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 b/sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 b/sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 b/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 deleted file mode 100644 index 916ac1482c061..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 +++ /dev/null @@ -1,18 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWTABLES 'shtb_*') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Table Operator: - Show Tables - database name: default - pattern: shtb_* - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 b/sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 b/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 deleted file mode 100644 index 19102815663d2..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 +++ /dev/null @@ -1 +0,0 @@ -foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c b/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c deleted file mode 100644 index b67b816ee4b45..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c +++ /dev/null @@ -1,2 +0,0 @@ -shtb_test1 -shtb_test2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 b/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 deleted file mode 100644 index 8f06e234b2a6e..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 +++ /dev/null @@ -1,18 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWTABLES 'shtb_test1|shtb_test2') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Table Operator: - Show Tables - database name: default - pattern: shtb_test1|shtb_test2 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d b/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d deleted file mode 100644 index b67b816ee4b45..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d +++ /dev/null @@ -1,2 +0,0 @@ -shtb_test1 -shtb_test2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 b/sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c b/sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 b/sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 b/sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa b/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa deleted file mode 100644 index bec424bb026e9..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa +++ /dev/null @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOW_TABLESTATUS `src` default) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 b/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 deleted file mode 100644 index 9392b7dc686f6..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:src -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733248000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 b/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 deleted file mode 100644 index 9392b7dc686f6..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:src -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733248000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d b/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d deleted file mode 100644 index 9392b7dc686f6..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d +++ /dev/null @@ -1,14 +0,0 @@ -tableName:src -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733248000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 b/sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 b/sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 b/sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c b/sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 b/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 deleted file mode 100644 index f8b64f6056ea7..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:srcpart -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/srcpart/ds=2008-04-08/hr=11 -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:true -partitionColumns:struct partition_columns { string ds, string hr} -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733249000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 b/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 deleted file mode 100644 index 9392b7dc686f6..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:src -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733248000 diff --git a/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 b/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 index ce1a3441a1bc0..d882eea53ca3c 100644 --- a/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 +++ b/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 @@ -1,6 +1,10 @@ - -last_modified_by ocquery -last_modified_time 1408598216 +numFiles 0 +last_modified_by marmbrus +last_modified_time 1413891337 tmp true -transient_lastDdlTime 1408598216 +transient_lastDdlTime 1413891337 +COLUMN_STATS_ACCURATE false +totalSize 0 +numRows -1 bar bar value +rawDataSize -1 diff --git a/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 b/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 deleted file mode 100644 index b590724bca78d..0000000000000 --- a/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 +++ /dev/null @@ -1,17 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWPARTITIONS srcpart) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Partitions Operator: - Show Partitions - table: srcpart - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a b/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a deleted file mode 100644 index e9c723bbd136e..0000000000000 --- a/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a +++ /dev/null @@ -1,4 +0,0 @@ -ds=2008-04-08/hr=11 -ds=2008-04-08/hr=12 -ds=2008-04-09/hr=11 -ds=2008-04-09/hr=12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 b/sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 b/sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a b/sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 b/sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 b/sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 b/sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 b/sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 b/sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 b/sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd b/sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 b/sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 b/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 index 872146532307a..19304c010452e 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 @@ -1 +1 @@ -2 12 2 22 2 12 \ No newline at end of file +2 12 2 22 2 12 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a b/sql/hive/src/test/resources/golden/skewjoinopt13-4-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a rename to sql/hive/src/test/resources/golden/skewjoinopt13-4-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 b/sql/hive/src/test/resources/golden/skewjoinopt13-6-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 rename to sql/hive/src/test/resources/golden/skewjoinopt13-6-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 b/sql/hive/src/test/resources/golden/skewjoinopt13-8-8eb53fb8f05a43ee377aa1c927857e7c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 rename to sql/hive/src/test/resources/golden/skewjoinopt13-8-8eb53fb8f05a43ee377aa1c927857e7c diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 b/sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 b/sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 b/sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 b/sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 b/sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 b/sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d b/sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 b/sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 b/sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e b/sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 b/sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 b/sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 b/sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa b/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa index ee1bb6b112381..6ca70c5267e65 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa @@ -3,4 +3,4 @@ 8 18 8 18 8 18 8 18 8 28 8 18 -8 28 8 18 \ No newline at end of file +8 28 8 18 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a b/sql/hive/src/test/resources/golden/skewjoinopt18-4-abf4b7f158999af331d5dbfddf32fa68 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a rename to sql/hive/src/test/resources/golden/skewjoinopt18-4-abf4b7f158999af331d5dbfddf32fa68 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 b/sql/hive/src/test/resources/golden/skewjoinopt18-8-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 rename to sql/hive/src/test/resources/golden/skewjoinopt18-8-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e b/sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 b/sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d b/sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 b/sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 b/sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb b/sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f b/sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 b/sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d b/sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 b/sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd b/sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 b/sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c b/sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd b/sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 b/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 index acd4039d35669..d3938a35d72f5 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 @@ -1,4 +1,4 @@ 2 1 2 22 3 1 3 13 8 2 8 18 -8 2 8 18 \ No newline at end of file +8 2 8 18 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 b/sql/hive/src/test/resources/golden/skewjoinopt9-4-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 rename to sql/hive/src/test/resources/golden/skewjoinopt9-4-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 b/sql/hive/src/test/resources/golden/skewjoinopt9-6-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 rename to sql/hive/src/test/resources/golden/skewjoinopt9-6-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d b/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d index f608d53f51ebe..826338ea56810 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d @@ -9,4 +9,4 @@ 8 28 8 18 8 28 8 18 8 28 8 18 -8 28 8 18 \ No newline at end of file +8 28 8 18 diff --git a/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 b/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 index 7a442f02e8d7a..1179e20c2847c 100644 --- a/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 +++ b/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 @@ -1,3 +1,3 @@ 0 2 0 2 -0 2 \ No newline at end of file +0 2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 index 836ee718649ad..da83658b68646 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 @@ -1 +1 @@ -51 val_3 51 val_30 \ No newline at end of file +51 val_3 51 val_30 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa b/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa index 8f358bffec51d..1ab49661a01f4 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa @@ -2,4 +2,4 @@ 51 val_3 51 val_30 52 val_4 NULL NULL 53 val_5 NULL NULL -49 val_10 NULL NULL \ No newline at end of file +49 val_10 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 index 6197cf72c3454..ba7969b99d79f 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 @@ -1,4 +1,4 @@ NULL NULL 50 val_20 NULL NULL 50 val_23 NULL NULL 50 val_25 -51 val_3 51 val_30 \ No newline at end of file +51 val_3 51 val_30 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 index d650d44f42404..ad8b511265e20 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 @@ -5,4 +5,4 @@ NULL NULL 50 val_23 NULL NULL 50 val_25 51 val_3 51 val_30 52 val_4 NULL NULL -53 val_5 NULL NULL \ No newline at end of file +53 val_5 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 index 836ee718649ad..da83658b68646 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 @@ -1 +1 @@ -51 val_3 51 val_30 \ No newline at end of file +51 val_3 51 val_30 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 index 8f358bffec51d..1ab49661a01f4 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 @@ -2,4 +2,4 @@ 51 val_3 51 val_30 52 val_4 NULL NULL 53 val_5 NULL NULL -49 val_10 NULL NULL \ No newline at end of file +49 val_10 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b b/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b index 6197cf72c3454..ba7969b99d79f 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b @@ -1,4 +1,4 @@ NULL NULL 50 val_20 NULL NULL 50 val_23 NULL NULL 50 val_25 -51 val_3 51 val_30 \ No newline at end of file +51 val_3 51 val_30 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae b/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae index d650d44f42404..ad8b511265e20 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae @@ -5,4 +5,4 @@ NULL NULL 50 val_23 NULL NULL 50 val_25 51 val_3 51 val_30 52 val_4 NULL NULL -53 val_5 NULL NULL \ No newline at end of file +53 val_5 NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-68d65d622e45f86d4a6c7d1d09ef823b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-3-68d65d622e45f86d4a6c7d1d09ef823b diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d31ad2289181131982ef3e9cd8c6386e similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d31ad2289181131982ef3e9cd8c6386e diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b b/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f7fb003fa65cadcd0b13cbdd7b355988 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f7fb003fa65cadcd0b13cbdd7b355988 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa b/sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af b/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb b/sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af b/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af deleted file mode 100644 index 8975db9a05036..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af +++ /dev/null @@ -1 +0,0 @@ -293 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b b/sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e deleted file mode 100644 index 8975db9a05036..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e +++ /dev/null @@ -1 +0,0 @@ -293 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 index b828077157966..9f4c46e548d04 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 @@ -7,4 +7,4 @@ 0 val_0 0 val_0 0 val_0 0 val_0 0 val_0 0 val_0 -2 val_2 2 val_2 \ No newline at end of file +2 val_2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d b/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a b/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a index 4a9735f855f96..ec7496a567609 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a @@ -3,4 +3,4 @@ 4 1 5 9 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 index 62f9457511f87..1e8b314962144 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 @@ -1 +1 @@ -6 \ No newline at end of file +6 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee b/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e b/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b b/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 index 2ebc6516c7df1..f6b91e0e1f8dd 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 @@ -1 +1 @@ -56 \ No newline at end of file +56 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a b/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a index b828077157966..9f4c46e548d04 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a @@ -7,4 +7,4 @@ 0 val_0 0 val_0 0 val_0 0 val_0 0 val_0 0 val_0 -2 val_2 2 val_2 \ No newline at end of file +2 val_2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f index 471d725e7bfa3..46d384b9dfb9b 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f @@ -7,4 +7,4 @@ 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 -2 2 val_2 2 2 val_2 \ No newline at end of file +2 2 val_2 2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e b/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e index 471d725e7bfa3..46d384b9dfb9b 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e @@ -7,4 +7,4 @@ 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 -2 2 val_2 2 2 val_2 \ No newline at end of file +2 2 val_2 2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 index 471d725e7bfa3..46d384b9dfb9b 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 @@ -7,4 +7,4 @@ 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 -2 2 val_2 2 2 val_2 \ No newline at end of file +2 2 val_2 2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf b/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf index 67d892c80f493..1f3d8a7a1fc08 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf @@ -1 +1 @@ -1028 \ No newline at end of file +1028 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a b/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a index 25398d9017c7f..a84e60c846ab2 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a @@ -1 +1 @@ -4378 \ No newline at end of file +4378 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 index 25398d9017c7f..a84e60c846ab2 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 @@ -1 +1 @@ -4378 \ No newline at end of file +4378 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc b/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc index 09b5b315bcf45..e0fa4e2d8601d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc @@ -1 +1 @@ -13126 \ No newline at end of file +13126 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d b/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a b/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab b/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe b/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 deleted file mode 100644 index dce6588ca1420..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 deleted file mode 100644 index 86ee83a4a2686..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 +++ /dev/null @@ -1 +0,0 @@ -40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab b/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab deleted file mode 100644 index d99e90eb9675f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab +++ /dev/null @@ -1 +0,0 @@ -29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc b/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc deleted file mode 100644 index dce6588ca1420..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec b/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec deleted file mode 100644 index 86ee83a4a2686..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec +++ /dev/null @@ -1 +0,0 @@ -40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e deleted file mode 100644 index d99e90eb9675f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e +++ /dev/null @@ -1 +0,0 @@ -29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b b/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b deleted file mode 100644 index dce6588ca1420..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 deleted file mode 100644 index 86ee83a4a2686..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 +++ /dev/null @@ -1 +0,0 @@ -40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f deleted file mode 100644 index d99e90eb9675f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f +++ /dev/null @@ -1 +0,0 @@ -29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb b/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb deleted file mode 100644 index dce6588ca1420..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 deleted file mode 100644 index 86ee83a4a2686..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 +++ /dev/null @@ -1 +0,0 @@ -40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f deleted file mode 100644 index d99e90eb9675f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f +++ /dev/null @@ -1 +0,0 @@ -29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d b/sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac b/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac index 7b5974818c085..4482f7ff91c62 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac @@ -4,4 +4,4 @@ 49 val_1 49 val_17 49 val_10 49 val_17 49 val_1 49 val_19 -49 val_10 49 val_19 \ No newline at end of file +49 val_10 49 val_19 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 index 0e1d132524064..cdacc0434caa7 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 @@ -6,4 +6,4 @@ 53 val_5 NULL NULL 49 val_10 49 val_10 49 val_10 49 val_17 -49 val_10 49 val_19 \ No newline at end of file +49 val_10 49 val_19 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 index 32be455fba8c4..37d71f5b522c6 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 @@ -6,4 +6,4 @@ 49 val_1 49 val_19 49 val_10 49 val_19 NULL NULL 50 val_20 -NULL NULL 50 val_23 \ No newline at end of file +NULL NULL 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca b/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca index 2496c4400b0b9..364a70c242916 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca @@ -8,4 +8,4 @@ NULL NULL 50 val_20 NULL NULL 50 val_23 51 val_3 NULL NULL 52 val_4 52 val_4 -53 val_5 NULL NULL \ No newline at end of file +53 val_5 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 index 7b5974818c085..4482f7ff91c62 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 @@ -4,4 +4,4 @@ 49 val_1 49 val_17 49 val_10 49 val_17 49 val_1 49 val_19 -49 val_10 49 val_19 \ No newline at end of file +49 val_10 49 val_19 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b b/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b index 0e1d132524064..cdacc0434caa7 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b @@ -6,4 +6,4 @@ 53 val_5 NULL NULL 49 val_10 49 val_10 49 val_10 49 val_17 -49 val_10 49 val_19 \ No newline at end of file +49 val_10 49 val_19 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 index 32be455fba8c4..37d71f5b522c6 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 @@ -6,4 +6,4 @@ 49 val_1 49 val_19 49 val_10 49 val_19 NULL NULL 50 val_20 -NULL NULL 50 val_23 \ No newline at end of file +NULL NULL 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 index 2496c4400b0b9..364a70c242916 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 @@ -8,4 +8,4 @@ NULL NULL 50 val_20 NULL NULL 50 val_23 51 val_3 NULL NULL 52 val_4 52 val_4 -53 val_5 NULL NULL \ No newline at end of file +53 val_5 NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab b/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d b/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d deleted file mode 100644 index b6e27607fb529..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d +++ /dev/null @@ -1 +0,0 @@ -242 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 deleted file mode 100644 index ce83bd94b3310..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 +++ /dev/null @@ -1 +0,0 @@ -258 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a b/sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a b/sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e b/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff b/sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a b/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe b/sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e b/sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad b/sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a b/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b b/sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb b/sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad b/sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc b/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-10-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f b/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-12-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f b/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d b/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff b/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 index 2c05a72679b22..8aa583680ba51 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 @@ -3,4 +3,4 @@ 50 val_25 50 val_20 50 val_20 50 val_23 50 val_23 50 val_23 -50 val_25 50 val_23 \ No newline at end of file +50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 index 0d9bce83ea87c..8f8addd11eabf 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 @@ -4,4 +4,4 @@ 50 val_23 50 val_23 50 val_25 50 val_20 50 val_25 50 val_23 -51 val_30 NULL NULL \ No newline at end of file +51 val_30 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e b/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e index d4c1adc92802d..c94cb185c6199 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e @@ -7,4 +7,4 @@ NULL NULL 49 val_19 50 val_25 50 val_20 50 val_20 50 val_23 50 val_23 50 val_23 -50 val_25 50 val_23 \ No newline at end of file +50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a b/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a index b62eec8583c92..fb499272e90c5 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a @@ -8,4 +8,4 @@ NULL NULL 49 val_19 50 val_25 50 val_20 50 val_25 50 val_23 51 val_30 NULL NULL -NULL NULL 52 val_4 \ No newline at end of file +NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 index 2c05a72679b22..8aa583680ba51 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 @@ -3,4 +3,4 @@ 50 val_25 50 val_20 50 val_20 50 val_23 50 val_23 50 val_23 -50 val_25 50 val_23 \ No newline at end of file +50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd b/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd index 0d9bce83ea87c..8f8addd11eabf 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd @@ -4,4 +4,4 @@ 50 val_23 50 val_23 50 val_25 50 val_20 50 val_25 50 val_23 -51 val_30 NULL NULL \ No newline at end of file +51 val_30 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 index d4c1adc92802d..c94cb185c6199 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 @@ -7,4 +7,4 @@ NULL NULL 49 val_19 50 val_25 50 val_20 50 val_20 50 val_23 50 val_23 50 val_23 -50 val_25 50 val_23 \ No newline at end of file +50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 index b62eec8583c92..fb499272e90c5 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 @@ -8,4 +8,4 @@ NULL NULL 49 val_19 50 val_25 50 val_20 50 val_25 50 val_23 51 val_30 NULL NULL -NULL NULL 52 val_4 \ No newline at end of file +NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea b/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d b/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d index 46af2ffd2b620..31c2549443bb1 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d @@ -2,4 +2,4 @@ 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL 53 val_5 NULL NULL NULL NULL -49 val_10 NULL NULL NULL NULL \ No newline at end of file +49 val_10 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 index 9047baeb236e5..fa5860b62f611 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 @@ -3,4 +3,4 @@ NULL NULL NULL NULL 49 val_10 NULL NULL NULL NULL 49 val_17 NULL NULL NULL NULL 49 val_19 NULL NULL NULL NULL 50 val_20 -NULL NULL NULL NULL 50 val_23 \ No newline at end of file +NULL NULL NULL NULL 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 index 47626758b4182..00e10b37e2fa1 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 @@ -8,4 +8,4 @@ NULL NULL NULL NULL 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL NULL NULL NULL NULL 52 val_4 -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 index 29616d6368661..84486137a37e4 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 @@ -3,4 +3,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_23 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 index 6f90cbf839656..e2c6bab43530d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 @@ -4,4 +4,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 -51 val_3 51 val_30 NULL NULL \ No newline at end of file +51 val_3 51 val_30 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 index 3b195f9dff935..8ea9abf42f678 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 @@ -7,4 +7,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_23 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e b/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e index e18ae75d14a37..f0bda77df9202 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e @@ -8,4 +8,4 @@ NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL -NULL NULL NULL NULL 52 val_4 \ No newline at end of file +NULL NULL NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 index 8e51f0864314a..0c9e2268914bd 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 @@ -3,4 +3,4 @@ NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b b/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f index 51676ca2c94e4..c9ca4763cbabe 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f @@ -8,4 +8,4 @@ NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 index 4a125291de271..812d839a1c642 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 @@ -7,4 +7,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 -NULL NULL NULL NULL 52 val_4 \ No newline at end of file +NULL NULL NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 index 660e9044e889a..c019550d25827 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 @@ -12,4 +12,4 @@ NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL NULL NULL NULL NULL 52 val_4 -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 index 46af2ffd2b620..31c2549443bb1 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 @@ -2,4 +2,4 @@ 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL 53 val_5 NULL NULL NULL NULL -49 val_10 NULL NULL NULL NULL \ No newline at end of file +49 val_10 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 index 9047baeb236e5..fa5860b62f611 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 @@ -3,4 +3,4 @@ NULL NULL NULL NULL 49 val_10 NULL NULL NULL NULL 49 val_17 NULL NULL NULL NULL 49 val_19 NULL NULL NULL NULL 50 val_20 -NULL NULL NULL NULL 50 val_23 \ No newline at end of file +NULL NULL NULL NULL 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc index 47626758b4182..00e10b37e2fa1 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc @@ -8,4 +8,4 @@ NULL NULL NULL NULL 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL NULL NULL NULL NULL 52 val_4 -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d b/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d index 29616d6368661..84486137a37e4 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d @@ -3,4 +3,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_23 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a b/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a index 6f90cbf839656..e2c6bab43530d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a @@ -4,4 +4,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 -51 val_3 51 val_30 NULL NULL \ No newline at end of file +51 val_3 51 val_30 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b b/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b index 3b195f9dff935..8ea9abf42f678 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b @@ -7,4 +7,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_23 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 index e18ae75d14a37..f0bda77df9202 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 @@ -8,4 +8,4 @@ NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL -NULL NULL NULL NULL 52 val_4 \ No newline at end of file +NULL NULL NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 index 8e51f0864314a..0c9e2268914bd 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 @@ -3,4 +3,4 @@ NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 index 51676ca2c94e4..c9ca4763cbabe 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 @@ -8,4 +8,4 @@ NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc index 4a125291de271..812d839a1c642 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc @@ -7,4 +7,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 -NULL NULL NULL NULL 52 val_4 \ No newline at end of file +NULL NULL NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 index 660e9044e889a..c019550d25827 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 @@ -12,4 +12,4 @@ NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL NULL NULL NULL NULL 52 val_4 -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca b/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca index c2e3ea8b0c8e2..b212e93a0a8c2 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca @@ -1025,4 +1025,4 @@ 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 -498 val_498 498 val_498 \ No newline at end of file +498 val_498 498 val_498 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e index 0e17d179f4167..11bd621866ba8 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e @@ -1 +1 @@ -278697 278697 101852390308 101852390308 \ No newline at end of file +278697 278697 101852390308 101852390308 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a index 0e17d179f4167..11bd621866ba8 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a @@ -1 +1 @@ -278697 278697 101852390308 101852390308 \ No newline at end of file +278697 278697 101852390308 101852390308 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca b/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca index c2e3ea8b0c8e2..b212e93a0a8c2 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca @@ -1025,4 +1025,4 @@ 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 -498 val_498 498 val_498 \ No newline at end of file +498 val_498 498 val_498 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e index 0e17d179f4167..11bd621866ba8 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e @@ -1 +1 @@ -278697 278697 101852390308 101852390308 \ No newline at end of file +278697 278697 101852390308 101852390308 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a index 0e17d179f4167..11bd621866ba8 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a @@ -1 +1 @@ -278697 278697 101852390308 101852390308 \ No newline at end of file +278697 278697 101852390308 101852390308 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 index 4cd5eefea2a45..dfdc3444cc072 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 @@ -497,4 +497,4 @@ NULL NULL 496 val_496 NULL NULL 497 val_497 NULL NULL 498 val_498 NULL NULL 498 val_498 -NULL NULL 498 val_498 \ No newline at end of file +NULL NULL 498 val_498 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a index 4cd5eefea2a45..dfdc3444cc072 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a @@ -497,4 +497,4 @@ NULL NULL 496 val_496 NULL NULL 497 val_497 NULL NULL 498 val_498 NULL NULL 498 val_498 -NULL NULL 498 val_498 \ No newline at end of file +NULL NULL 498 val_498 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e index ca97f45c90026..2e711200bae28 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e @@ -1 +1 @@ -0 130091 0 36210398070 \ No newline at end of file +0 130091 0 36210398070 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a index ca97f45c90026..2e711200bae28 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a @@ -1 +1 @@ -0 130091 0 36210398070 \ No newline at end of file +0 130091 0 36210398070 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff b/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff index ca97f45c90026..2e711200bae28 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff @@ -1 +1 @@ -0 130091 0 36210398070 \ No newline at end of file +0 130091 0 36210398070 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb b/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac b/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-f983875c44b290b0884a22b6be6adc8 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac rename to sql/hive/src/test/resources/golden/smb_mapjoin_7-8-f983875c44b290b0884a22b6be6adc8 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c b/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-84a394d962965e38593883742cc32c0d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c rename to sql/hive/src/test/resources/golden/smb_mapjoin_7-9-84a394d962965e38593883742cc32c0d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-eee18fc4192a4aa92a066eb66513be93 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 rename to sql/hive/src/test/resources/golden/smb_mapjoin_8-5-eee18fc4192a4aa92a066eb66513be93 diff --git a/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 b/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 +++ b/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a index 67d892c80f493..1f3d8a7a1fc08 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a @@ -1 +1 @@ -1028 \ No newline at end of file +1028 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a index 67d892c80f493..1f3d8a7a1fc08 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a @@ -1 +1 @@ -1028 \ No newline at end of file +1028 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 b/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 index 688eea009d292..d3a7b34f283b1 100644 --- a/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 +++ b/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1390903702, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390903702, numRows=500, totalSize=5812, rawDataSize=5312}) +Detailed Partition Information Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1413893941, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413893941, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}) diff --git a/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c b/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c index 4d3a3d8f237b3..965913df89c64 100644 --- a/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c +++ b/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1390903694, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, p3=v3, transient_lastDdlTime=1390903702, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1413893934, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413893934}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c b/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c index a18ab93992cc5..b34342d7815e1 100644 --- a/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c +++ b/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c @@ -1,4 +1,4 @@ -key string None -value string None +key string +value string -Detailed Table Information Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1390903702, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, p3=v3, transient_lastDdlTime=1390903710, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1413893941, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413893948, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 b/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 index d861060f13cae..c1e56cd3180cb 100644 --- a/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 +++ b/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1390903719, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390903719, numRows=500, totalSize=5812, rawDataSize=5312}) +Detailed Partition Information Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1413893955, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413893955, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}) diff --git a/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c b/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c index aa4dfe1e5ba39..25d227999c390 100644 --- a/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c +++ b/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1390903710, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, p3=v3, transient_lastDdlTime=1390903719, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1413893948, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413893948}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c b/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c index 8a5ae7cda1c57..8b503b045f6ec 100644 --- a/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c +++ b/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c @@ -1,4 +1,4 @@ -key string None -value string None +key string +value string -Detailed Table Information Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1390903686, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, p3=v3, transient_lastDdlTime=1390903694, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1413893927, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413893934, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e b/sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b b/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b deleted file mode 100644 index f87b67cb2006f..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:17:02 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_t1 -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 4 - numPartitions 4 - numRows 2000 - p3 v3 - rawDataSize 21248 - totalSize 23248 - transient_lastDdlTime 1390904237 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 b/sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 b/sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 b/sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 b/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 deleted file mode 100644 index 19a5e1e042ae7..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 +++ /dev/null @@ -1,34 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:17:02 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_t1 -Table Type: MANAGED_TABLE -Table Parameters: - p3 v3 - transient_lastDdlTime 1390904222 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e b/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e deleted file mode 100644 index 7ddea146475d1..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_t1) (TOK_PARTSPEC (TOK_PARTVAL ds) (TOK_PARTVAL hr)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_t1 - TableScan - alias: analyze_t1 - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 b/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b b/sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a b/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a deleted file mode 100644 index 5c8878cba5e71..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a +++ /dev/null @@ -1,37 +0,0 @@ -# col_name data_type comment - -key string None -value string None - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 15:57:53 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2617911713640090101/stats_partitioned -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390953481 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 b/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa b/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa deleted file mode 100644 index d9ba3814de367..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa +++ /dev/null @@ -1,37 +0,0 @@ -# col_name data_type comment - -key string None -value string None - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 15:57:53 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2617911713640090101/stats_partitioned -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 1 - numRows 500 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1390953492 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 b/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 deleted file mode 100644 index bf2a7f452edc3..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 +++ /dev/null @@ -1,6 +0,0 @@ -1 test_part test_Part -2 test_part test_Part -3 test_part test_Part -4 test_part test_Part -5 test_part test_Part -6 test_part test_Part diff --git a/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc b/sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 b/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 deleted file mode 100644 index bf2a7f452edc3..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 +++ /dev/null @@ -1,6 +0,0 @@ -1 test_part test_Part -2 test_part test_Part -3 test_part test_Part -4 test_part test_Part -5 test_part test_Part -6 test_part test_Part diff --git a/sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 b/sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b b/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b deleted file mode 100644 index cb1c9def371df..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b +++ /dev/null @@ -1,38 +0,0 @@ -# col_name data_type comment - -col1 string None - -# Partition Information -# col_name data_type comment - -pcol1 string None -pcol2 string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:17:38 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_dst -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 1 - numRows 6 - p3 v3 - rawDataSize 6 - totalSize 171 - transient_lastDdlTime 1390904285 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 b/sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 b/sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 b/sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 b/sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 b/sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 b/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 deleted file mode 100644 index 8bc7c14574497..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 +++ /dev/null @@ -1,43 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_LOAD '/Users/marmbrus/workspace/hive/data/files/test.dat' (TOK_TAB (TOK_TABNAME hive_test_src)) local overwrite) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - Stage-2 depends on stages: Stage-1 - -STAGE PLANS: - Stage: Stage-0 - Copy - source: file:/Users/marmbrus/workspace/hive/data/files/test.dat - destination: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10000 - - Stage: Stage-1 - Move Operator - tables: - replace: true - source: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10000 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns col1 - columns.types string - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_src - name default.hive_test_src - p3 v3 - serialization.ddl struct hive_test_src { string col1} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - transient_lastDdlTime 1390904258 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.hive_test_src - tmp directory: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10001 - - Stage: Stage-2 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c b/sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d b/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d deleted file mode 100644 index 4916ef91b0866..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -col1 string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:17:38 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_src -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 0 - numRows 0 - p3 v3 - rawDataSize 0 - totalSize 11 - transient_lastDdlTime 1390904258 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 b/sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 b/sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 b/sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 b/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 deleted file mode 100644 index 1634adfc4d70d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 +++ /dev/null @@ -1,2 +0,0 @@ -ds=2008-12-31/hr=11 -ds=2008-12-31/hr=12 diff --git a/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 b/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 deleted file mode 100644 index d7a8f25b41301..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 +++ /dev/null @@ -1,1000 +0,0 @@ -238 val_238 2008-04-08 11 -86 val_86 2008-04-08 11 -311 val_311 2008-04-08 11 -27 val_27 2008-04-08 11 -165 val_165 2008-04-08 11 -409 val_409 2008-04-08 11 -255 val_255 2008-04-08 11 -278 val_278 2008-04-08 11 -98 val_98 2008-04-08 11 -484 val_484 2008-04-08 11 -265 val_265 2008-04-08 11 -193 val_193 2008-04-08 11 -401 val_401 2008-04-08 11 -150 val_150 2008-04-08 11 -273 val_273 2008-04-08 11 -224 val_224 2008-04-08 11 -369 val_369 2008-04-08 11 -66 val_66 2008-04-08 11 -128 val_128 2008-04-08 11 -213 val_213 2008-04-08 11 -146 val_146 2008-04-08 11 -406 val_406 2008-04-08 11 -429 val_429 2008-04-08 11 -374 val_374 2008-04-08 11 -152 val_152 2008-04-08 11 -469 val_469 2008-04-08 11 -145 val_145 2008-04-08 11 -495 val_495 2008-04-08 11 -37 val_37 2008-04-08 11 -327 val_327 2008-04-08 11 -281 val_281 2008-04-08 11 -277 val_277 2008-04-08 11 -209 val_209 2008-04-08 11 -15 val_15 2008-04-08 11 -82 val_82 2008-04-08 11 -403 val_403 2008-04-08 11 -166 val_166 2008-04-08 11 -417 val_417 2008-04-08 11 -430 val_430 2008-04-08 11 -252 val_252 2008-04-08 11 -292 val_292 2008-04-08 11 -219 val_219 2008-04-08 11 -287 val_287 2008-04-08 11 -153 val_153 2008-04-08 11 -193 val_193 2008-04-08 11 -338 val_338 2008-04-08 11 -446 val_446 2008-04-08 11 -459 val_459 2008-04-08 11 -394 val_394 2008-04-08 11 -237 val_237 2008-04-08 11 -482 val_482 2008-04-08 11 -174 val_174 2008-04-08 11 -413 val_413 2008-04-08 11 -494 val_494 2008-04-08 11 -207 val_207 2008-04-08 11 -199 val_199 2008-04-08 11 -466 val_466 2008-04-08 11 -208 val_208 2008-04-08 11 -174 val_174 2008-04-08 11 -399 val_399 2008-04-08 11 -396 val_396 2008-04-08 11 -247 val_247 2008-04-08 11 -417 val_417 2008-04-08 11 -489 val_489 2008-04-08 11 -162 val_162 2008-04-08 11 -377 val_377 2008-04-08 11 -397 val_397 2008-04-08 11 -309 val_309 2008-04-08 11 -365 val_365 2008-04-08 11 -266 val_266 2008-04-08 11 -439 val_439 2008-04-08 11 -342 val_342 2008-04-08 11 -367 val_367 2008-04-08 11 -325 val_325 2008-04-08 11 -167 val_167 2008-04-08 11 -195 val_195 2008-04-08 11 -475 val_475 2008-04-08 11 -17 val_17 2008-04-08 11 -113 val_113 2008-04-08 11 -155 val_155 2008-04-08 11 -203 val_203 2008-04-08 11 -339 val_339 2008-04-08 11 -0 val_0 2008-04-08 11 -455 val_455 2008-04-08 11 -128 val_128 2008-04-08 11 -311 val_311 2008-04-08 11 -316 val_316 2008-04-08 11 -57 val_57 2008-04-08 11 -302 val_302 2008-04-08 11 -205 val_205 2008-04-08 11 -149 val_149 2008-04-08 11 -438 val_438 2008-04-08 11 -345 val_345 2008-04-08 11 -129 val_129 2008-04-08 11 -170 val_170 2008-04-08 11 -20 val_20 2008-04-08 11 -489 val_489 2008-04-08 11 -157 val_157 2008-04-08 11 -378 val_378 2008-04-08 11 -221 val_221 2008-04-08 11 -92 val_92 2008-04-08 11 -111 val_111 2008-04-08 11 -47 val_47 2008-04-08 11 -72 val_72 2008-04-08 11 -4 val_4 2008-04-08 11 -280 val_280 2008-04-08 11 -35 val_35 2008-04-08 11 -427 val_427 2008-04-08 11 -277 val_277 2008-04-08 11 -208 val_208 2008-04-08 11 -356 val_356 2008-04-08 11 -399 val_399 2008-04-08 11 -169 val_169 2008-04-08 11 -382 val_382 2008-04-08 11 -498 val_498 2008-04-08 11 -125 val_125 2008-04-08 11 -386 val_386 2008-04-08 11 -437 val_437 2008-04-08 11 -469 val_469 2008-04-08 11 -192 val_192 2008-04-08 11 -286 val_286 2008-04-08 11 -187 val_187 2008-04-08 11 -176 val_176 2008-04-08 11 -54 val_54 2008-04-08 11 -459 val_459 2008-04-08 11 -51 val_51 2008-04-08 11 -138 val_138 2008-04-08 11 -103 val_103 2008-04-08 11 -239 val_239 2008-04-08 11 -213 val_213 2008-04-08 11 -216 val_216 2008-04-08 11 -430 val_430 2008-04-08 11 -278 val_278 2008-04-08 11 -176 val_176 2008-04-08 11 -289 val_289 2008-04-08 11 -221 val_221 2008-04-08 11 -65 val_65 2008-04-08 11 -318 val_318 2008-04-08 11 -332 val_332 2008-04-08 11 -311 val_311 2008-04-08 11 -275 val_275 2008-04-08 11 -137 val_137 2008-04-08 11 -241 val_241 2008-04-08 11 -83 val_83 2008-04-08 11 -333 val_333 2008-04-08 11 -180 val_180 2008-04-08 11 -284 val_284 2008-04-08 11 -12 val_12 2008-04-08 11 -230 val_230 2008-04-08 11 -181 val_181 2008-04-08 11 -67 val_67 2008-04-08 11 -260 val_260 2008-04-08 11 -404 val_404 2008-04-08 11 -384 val_384 2008-04-08 11 -489 val_489 2008-04-08 11 -353 val_353 2008-04-08 11 -373 val_373 2008-04-08 11 -272 val_272 2008-04-08 11 -138 val_138 2008-04-08 11 -217 val_217 2008-04-08 11 -84 val_84 2008-04-08 11 -348 val_348 2008-04-08 11 -466 val_466 2008-04-08 11 -58 val_58 2008-04-08 11 -8 val_8 2008-04-08 11 -411 val_411 2008-04-08 11 -230 val_230 2008-04-08 11 -208 val_208 2008-04-08 11 -348 val_348 2008-04-08 11 -24 val_24 2008-04-08 11 -463 val_463 2008-04-08 11 -431 val_431 2008-04-08 11 -179 val_179 2008-04-08 11 -172 val_172 2008-04-08 11 -42 val_42 2008-04-08 11 -129 val_129 2008-04-08 11 -158 val_158 2008-04-08 11 -119 val_119 2008-04-08 11 -496 val_496 2008-04-08 11 -0 val_0 2008-04-08 11 -322 val_322 2008-04-08 11 -197 val_197 2008-04-08 11 -468 val_468 2008-04-08 11 -393 val_393 2008-04-08 11 -454 val_454 2008-04-08 11 -100 val_100 2008-04-08 11 -298 val_298 2008-04-08 11 -199 val_199 2008-04-08 11 -191 val_191 2008-04-08 11 -418 val_418 2008-04-08 11 -96 val_96 2008-04-08 11 -26 val_26 2008-04-08 11 -165 val_165 2008-04-08 11 -327 val_327 2008-04-08 11 -230 val_230 2008-04-08 11 -205 val_205 2008-04-08 11 -120 val_120 2008-04-08 11 -131 val_131 2008-04-08 11 -51 val_51 2008-04-08 11 -404 val_404 2008-04-08 11 -43 val_43 2008-04-08 11 -436 val_436 2008-04-08 11 -156 val_156 2008-04-08 11 -469 val_469 2008-04-08 11 -468 val_468 2008-04-08 11 -308 val_308 2008-04-08 11 -95 val_95 2008-04-08 11 -196 val_196 2008-04-08 11 -288 val_288 2008-04-08 11 -481 val_481 2008-04-08 11 -457 val_457 2008-04-08 11 -98 val_98 2008-04-08 11 -282 val_282 2008-04-08 11 -197 val_197 2008-04-08 11 -187 val_187 2008-04-08 11 -318 val_318 2008-04-08 11 -318 val_318 2008-04-08 11 -409 val_409 2008-04-08 11 -470 val_470 2008-04-08 11 -137 val_137 2008-04-08 11 -369 val_369 2008-04-08 11 -316 val_316 2008-04-08 11 -169 val_169 2008-04-08 11 -413 val_413 2008-04-08 11 -85 val_85 2008-04-08 11 -77 val_77 2008-04-08 11 -0 val_0 2008-04-08 11 -490 val_490 2008-04-08 11 -87 val_87 2008-04-08 11 -364 val_364 2008-04-08 11 -179 val_179 2008-04-08 11 -118 val_118 2008-04-08 11 -134 val_134 2008-04-08 11 -395 val_395 2008-04-08 11 -282 val_282 2008-04-08 11 -138 val_138 2008-04-08 11 -238 val_238 2008-04-08 11 -419 val_419 2008-04-08 11 -15 val_15 2008-04-08 11 -118 val_118 2008-04-08 11 -72 val_72 2008-04-08 11 -90 val_90 2008-04-08 11 -307 val_307 2008-04-08 11 -19 val_19 2008-04-08 11 -435 val_435 2008-04-08 11 -10 val_10 2008-04-08 11 -277 val_277 2008-04-08 11 -273 val_273 2008-04-08 11 -306 val_306 2008-04-08 11 -224 val_224 2008-04-08 11 -309 val_309 2008-04-08 11 -389 val_389 2008-04-08 11 -327 val_327 2008-04-08 11 -242 val_242 2008-04-08 11 -369 val_369 2008-04-08 11 -392 val_392 2008-04-08 11 -272 val_272 2008-04-08 11 -331 val_331 2008-04-08 11 -401 val_401 2008-04-08 11 -242 val_242 2008-04-08 11 -452 val_452 2008-04-08 11 -177 val_177 2008-04-08 11 -226 val_226 2008-04-08 11 -5 val_5 2008-04-08 11 -497 val_497 2008-04-08 11 -402 val_402 2008-04-08 11 -396 val_396 2008-04-08 11 -317 val_317 2008-04-08 11 -395 val_395 2008-04-08 11 -58 val_58 2008-04-08 11 -35 val_35 2008-04-08 11 -336 val_336 2008-04-08 11 -95 val_95 2008-04-08 11 -11 val_11 2008-04-08 11 -168 val_168 2008-04-08 11 -34 val_34 2008-04-08 11 -229 val_229 2008-04-08 11 -233 val_233 2008-04-08 11 -143 val_143 2008-04-08 11 -472 val_472 2008-04-08 11 -322 val_322 2008-04-08 11 -498 val_498 2008-04-08 11 -160 val_160 2008-04-08 11 -195 val_195 2008-04-08 11 -42 val_42 2008-04-08 11 -321 val_321 2008-04-08 11 -430 val_430 2008-04-08 11 -119 val_119 2008-04-08 11 -489 val_489 2008-04-08 11 -458 val_458 2008-04-08 11 -78 val_78 2008-04-08 11 -76 val_76 2008-04-08 11 -41 val_41 2008-04-08 11 -223 val_223 2008-04-08 11 -492 val_492 2008-04-08 11 -149 val_149 2008-04-08 11 -449 val_449 2008-04-08 11 -218 val_218 2008-04-08 11 -228 val_228 2008-04-08 11 -138 val_138 2008-04-08 11 -453 val_453 2008-04-08 11 -30 val_30 2008-04-08 11 -209 val_209 2008-04-08 11 -64 val_64 2008-04-08 11 -468 val_468 2008-04-08 11 -76 val_76 2008-04-08 11 -74 val_74 2008-04-08 11 -342 val_342 2008-04-08 11 -69 val_69 2008-04-08 11 -230 val_230 2008-04-08 11 -33 val_33 2008-04-08 11 -368 val_368 2008-04-08 11 -103 val_103 2008-04-08 11 -296 val_296 2008-04-08 11 -113 val_113 2008-04-08 11 -216 val_216 2008-04-08 11 -367 val_367 2008-04-08 11 -344 val_344 2008-04-08 11 -167 val_167 2008-04-08 11 -274 val_274 2008-04-08 11 -219 val_219 2008-04-08 11 -239 val_239 2008-04-08 11 -485 val_485 2008-04-08 11 -116 val_116 2008-04-08 11 -223 val_223 2008-04-08 11 -256 val_256 2008-04-08 11 -263 val_263 2008-04-08 11 -70 val_70 2008-04-08 11 -487 val_487 2008-04-08 11 -480 val_480 2008-04-08 11 -401 val_401 2008-04-08 11 -288 val_288 2008-04-08 11 -191 val_191 2008-04-08 11 -5 val_5 2008-04-08 11 -244 val_244 2008-04-08 11 -438 val_438 2008-04-08 11 -128 val_128 2008-04-08 11 -467 val_467 2008-04-08 11 -432 val_432 2008-04-08 11 -202 val_202 2008-04-08 11 -316 val_316 2008-04-08 11 -229 val_229 2008-04-08 11 -469 val_469 2008-04-08 11 -463 val_463 2008-04-08 11 -280 val_280 2008-04-08 11 -2 val_2 2008-04-08 11 -35 val_35 2008-04-08 11 -283 val_283 2008-04-08 11 -331 val_331 2008-04-08 11 -235 val_235 2008-04-08 11 -80 val_80 2008-04-08 11 -44 val_44 2008-04-08 11 -193 val_193 2008-04-08 11 -321 val_321 2008-04-08 11 -335 val_335 2008-04-08 11 -104 val_104 2008-04-08 11 -466 val_466 2008-04-08 11 -366 val_366 2008-04-08 11 -175 val_175 2008-04-08 11 -403 val_403 2008-04-08 11 -483 val_483 2008-04-08 11 -53 val_53 2008-04-08 11 -105 val_105 2008-04-08 11 -257 val_257 2008-04-08 11 -406 val_406 2008-04-08 11 -409 val_409 2008-04-08 11 -190 val_190 2008-04-08 11 -406 val_406 2008-04-08 11 -401 val_401 2008-04-08 11 -114 val_114 2008-04-08 11 -258 val_258 2008-04-08 11 -90 val_90 2008-04-08 11 -203 val_203 2008-04-08 11 -262 val_262 2008-04-08 11 -348 val_348 2008-04-08 11 -424 val_424 2008-04-08 11 -12 val_12 2008-04-08 11 -396 val_396 2008-04-08 11 -201 val_201 2008-04-08 11 -217 val_217 2008-04-08 11 -164 val_164 2008-04-08 11 -431 val_431 2008-04-08 11 -454 val_454 2008-04-08 11 -478 val_478 2008-04-08 11 -298 val_298 2008-04-08 11 -125 val_125 2008-04-08 11 -431 val_431 2008-04-08 11 -164 val_164 2008-04-08 11 -424 val_424 2008-04-08 11 -187 val_187 2008-04-08 11 -382 val_382 2008-04-08 11 -5 val_5 2008-04-08 11 -70 val_70 2008-04-08 11 -397 val_397 2008-04-08 11 -480 val_480 2008-04-08 11 -291 val_291 2008-04-08 11 -24 val_24 2008-04-08 11 -351 val_351 2008-04-08 11 -255 val_255 2008-04-08 11 -104 val_104 2008-04-08 11 -70 val_70 2008-04-08 11 -163 val_163 2008-04-08 11 -438 val_438 2008-04-08 11 -119 val_119 2008-04-08 11 -414 val_414 2008-04-08 11 -200 val_200 2008-04-08 11 -491 val_491 2008-04-08 11 -237 val_237 2008-04-08 11 -439 val_439 2008-04-08 11 -360 val_360 2008-04-08 11 -248 val_248 2008-04-08 11 -479 val_479 2008-04-08 11 -305 val_305 2008-04-08 11 -417 val_417 2008-04-08 11 -199 val_199 2008-04-08 11 -444 val_444 2008-04-08 11 -120 val_120 2008-04-08 11 -429 val_429 2008-04-08 11 -169 val_169 2008-04-08 11 -443 val_443 2008-04-08 11 -323 val_323 2008-04-08 11 -325 val_325 2008-04-08 11 -277 val_277 2008-04-08 11 -230 val_230 2008-04-08 11 -478 val_478 2008-04-08 11 -178 val_178 2008-04-08 11 -468 val_468 2008-04-08 11 -310 val_310 2008-04-08 11 -317 val_317 2008-04-08 11 -333 val_333 2008-04-08 11 -493 val_493 2008-04-08 11 -460 val_460 2008-04-08 11 -207 val_207 2008-04-08 11 -249 val_249 2008-04-08 11 -265 val_265 2008-04-08 11 -480 val_480 2008-04-08 11 -83 val_83 2008-04-08 11 -136 val_136 2008-04-08 11 -353 val_353 2008-04-08 11 -172 val_172 2008-04-08 11 -214 val_214 2008-04-08 11 -462 val_462 2008-04-08 11 -233 val_233 2008-04-08 11 -406 val_406 2008-04-08 11 -133 val_133 2008-04-08 11 -175 val_175 2008-04-08 11 -189 val_189 2008-04-08 11 -454 val_454 2008-04-08 11 -375 val_375 2008-04-08 11 -401 val_401 2008-04-08 11 -421 val_421 2008-04-08 11 -407 val_407 2008-04-08 11 -384 val_384 2008-04-08 11 -256 val_256 2008-04-08 11 -26 val_26 2008-04-08 11 -134 val_134 2008-04-08 11 -67 val_67 2008-04-08 11 -384 val_384 2008-04-08 11 -379 val_379 2008-04-08 11 -18 val_18 2008-04-08 11 -462 val_462 2008-04-08 11 -492 val_492 2008-04-08 11 -100 val_100 2008-04-08 11 -298 val_298 2008-04-08 11 -9 val_9 2008-04-08 11 -341 val_341 2008-04-08 11 -498 val_498 2008-04-08 11 -146 val_146 2008-04-08 11 -458 val_458 2008-04-08 11 -362 val_362 2008-04-08 11 -186 val_186 2008-04-08 11 -285 val_285 2008-04-08 11 -348 val_348 2008-04-08 11 -167 val_167 2008-04-08 11 -18 val_18 2008-04-08 11 -273 val_273 2008-04-08 11 -183 val_183 2008-04-08 11 -281 val_281 2008-04-08 11 -344 val_344 2008-04-08 11 -97 val_97 2008-04-08 11 -469 val_469 2008-04-08 11 -315 val_315 2008-04-08 11 -84 val_84 2008-04-08 11 -28 val_28 2008-04-08 11 -37 val_37 2008-04-08 11 -448 val_448 2008-04-08 11 -152 val_152 2008-04-08 11 -348 val_348 2008-04-08 11 -307 val_307 2008-04-08 11 -194 val_194 2008-04-08 11 -414 val_414 2008-04-08 11 -477 val_477 2008-04-08 11 -222 val_222 2008-04-08 11 -126 val_126 2008-04-08 11 -90 val_90 2008-04-08 11 -169 val_169 2008-04-08 11 -403 val_403 2008-04-08 11 -400 val_400 2008-04-08 11 -200 val_200 2008-04-08 11 -97 val_97 2008-04-08 11 -238 val_238 2008-04-08 12 -86 val_86 2008-04-08 12 -311 val_311 2008-04-08 12 -27 val_27 2008-04-08 12 -165 val_165 2008-04-08 12 -409 val_409 2008-04-08 12 -255 val_255 2008-04-08 12 -278 val_278 2008-04-08 12 -98 val_98 2008-04-08 12 -484 val_484 2008-04-08 12 -265 val_265 2008-04-08 12 -193 val_193 2008-04-08 12 -401 val_401 2008-04-08 12 -150 val_150 2008-04-08 12 -273 val_273 2008-04-08 12 -224 val_224 2008-04-08 12 -369 val_369 2008-04-08 12 -66 val_66 2008-04-08 12 -128 val_128 2008-04-08 12 -213 val_213 2008-04-08 12 -146 val_146 2008-04-08 12 -406 val_406 2008-04-08 12 -429 val_429 2008-04-08 12 -374 val_374 2008-04-08 12 -152 val_152 2008-04-08 12 -469 val_469 2008-04-08 12 -145 val_145 2008-04-08 12 -495 val_495 2008-04-08 12 -37 val_37 2008-04-08 12 -327 val_327 2008-04-08 12 -281 val_281 2008-04-08 12 -277 val_277 2008-04-08 12 -209 val_209 2008-04-08 12 -15 val_15 2008-04-08 12 -82 val_82 2008-04-08 12 -403 val_403 2008-04-08 12 -166 val_166 2008-04-08 12 -417 val_417 2008-04-08 12 -430 val_430 2008-04-08 12 -252 val_252 2008-04-08 12 -292 val_292 2008-04-08 12 -219 val_219 2008-04-08 12 -287 val_287 2008-04-08 12 -153 val_153 2008-04-08 12 -193 val_193 2008-04-08 12 -338 val_338 2008-04-08 12 -446 val_446 2008-04-08 12 -459 val_459 2008-04-08 12 -394 val_394 2008-04-08 12 -237 val_237 2008-04-08 12 -482 val_482 2008-04-08 12 -174 val_174 2008-04-08 12 -413 val_413 2008-04-08 12 -494 val_494 2008-04-08 12 -207 val_207 2008-04-08 12 -199 val_199 2008-04-08 12 -466 val_466 2008-04-08 12 -208 val_208 2008-04-08 12 -174 val_174 2008-04-08 12 -399 val_399 2008-04-08 12 -396 val_396 2008-04-08 12 -247 val_247 2008-04-08 12 -417 val_417 2008-04-08 12 -489 val_489 2008-04-08 12 -162 val_162 2008-04-08 12 -377 val_377 2008-04-08 12 -397 val_397 2008-04-08 12 -309 val_309 2008-04-08 12 -365 val_365 2008-04-08 12 -266 val_266 2008-04-08 12 -439 val_439 2008-04-08 12 -342 val_342 2008-04-08 12 -367 val_367 2008-04-08 12 -325 val_325 2008-04-08 12 -167 val_167 2008-04-08 12 -195 val_195 2008-04-08 12 -475 val_475 2008-04-08 12 -17 val_17 2008-04-08 12 -113 val_113 2008-04-08 12 -155 val_155 2008-04-08 12 -203 val_203 2008-04-08 12 -339 val_339 2008-04-08 12 -0 val_0 2008-04-08 12 -455 val_455 2008-04-08 12 -128 val_128 2008-04-08 12 -311 val_311 2008-04-08 12 -316 val_316 2008-04-08 12 -57 val_57 2008-04-08 12 -302 val_302 2008-04-08 12 -205 val_205 2008-04-08 12 -149 val_149 2008-04-08 12 -438 val_438 2008-04-08 12 -345 val_345 2008-04-08 12 -129 val_129 2008-04-08 12 -170 val_170 2008-04-08 12 -20 val_20 2008-04-08 12 -489 val_489 2008-04-08 12 -157 val_157 2008-04-08 12 -378 val_378 2008-04-08 12 -221 val_221 2008-04-08 12 -92 val_92 2008-04-08 12 -111 val_111 2008-04-08 12 -47 val_47 2008-04-08 12 -72 val_72 2008-04-08 12 -4 val_4 2008-04-08 12 -280 val_280 2008-04-08 12 -35 val_35 2008-04-08 12 -427 val_427 2008-04-08 12 -277 val_277 2008-04-08 12 -208 val_208 2008-04-08 12 -356 val_356 2008-04-08 12 -399 val_399 2008-04-08 12 -169 val_169 2008-04-08 12 -382 val_382 2008-04-08 12 -498 val_498 2008-04-08 12 -125 val_125 2008-04-08 12 -386 val_386 2008-04-08 12 -437 val_437 2008-04-08 12 -469 val_469 2008-04-08 12 -192 val_192 2008-04-08 12 -286 val_286 2008-04-08 12 -187 val_187 2008-04-08 12 -176 val_176 2008-04-08 12 -54 val_54 2008-04-08 12 -459 val_459 2008-04-08 12 -51 val_51 2008-04-08 12 -138 val_138 2008-04-08 12 -103 val_103 2008-04-08 12 -239 val_239 2008-04-08 12 -213 val_213 2008-04-08 12 -216 val_216 2008-04-08 12 -430 val_430 2008-04-08 12 -278 val_278 2008-04-08 12 -176 val_176 2008-04-08 12 -289 val_289 2008-04-08 12 -221 val_221 2008-04-08 12 -65 val_65 2008-04-08 12 -318 val_318 2008-04-08 12 -332 val_332 2008-04-08 12 -311 val_311 2008-04-08 12 -275 val_275 2008-04-08 12 -137 val_137 2008-04-08 12 -241 val_241 2008-04-08 12 -83 val_83 2008-04-08 12 -333 val_333 2008-04-08 12 -180 val_180 2008-04-08 12 -284 val_284 2008-04-08 12 -12 val_12 2008-04-08 12 -230 val_230 2008-04-08 12 -181 val_181 2008-04-08 12 -67 val_67 2008-04-08 12 -260 val_260 2008-04-08 12 -404 val_404 2008-04-08 12 -384 val_384 2008-04-08 12 -489 val_489 2008-04-08 12 -353 val_353 2008-04-08 12 -373 val_373 2008-04-08 12 -272 val_272 2008-04-08 12 -138 val_138 2008-04-08 12 -217 val_217 2008-04-08 12 -84 val_84 2008-04-08 12 -348 val_348 2008-04-08 12 -466 val_466 2008-04-08 12 -58 val_58 2008-04-08 12 -8 val_8 2008-04-08 12 -411 val_411 2008-04-08 12 -230 val_230 2008-04-08 12 -208 val_208 2008-04-08 12 -348 val_348 2008-04-08 12 -24 val_24 2008-04-08 12 -463 val_463 2008-04-08 12 -431 val_431 2008-04-08 12 -179 val_179 2008-04-08 12 -172 val_172 2008-04-08 12 -42 val_42 2008-04-08 12 -129 val_129 2008-04-08 12 -158 val_158 2008-04-08 12 -119 val_119 2008-04-08 12 -496 val_496 2008-04-08 12 -0 val_0 2008-04-08 12 -322 val_322 2008-04-08 12 -197 val_197 2008-04-08 12 -468 val_468 2008-04-08 12 -393 val_393 2008-04-08 12 -454 val_454 2008-04-08 12 -100 val_100 2008-04-08 12 -298 val_298 2008-04-08 12 -199 val_199 2008-04-08 12 -191 val_191 2008-04-08 12 -418 val_418 2008-04-08 12 -96 val_96 2008-04-08 12 -26 val_26 2008-04-08 12 -165 val_165 2008-04-08 12 -327 val_327 2008-04-08 12 -230 val_230 2008-04-08 12 -205 val_205 2008-04-08 12 -120 val_120 2008-04-08 12 -131 val_131 2008-04-08 12 -51 val_51 2008-04-08 12 -404 val_404 2008-04-08 12 -43 val_43 2008-04-08 12 -436 val_436 2008-04-08 12 -156 val_156 2008-04-08 12 -469 val_469 2008-04-08 12 -468 val_468 2008-04-08 12 -308 val_308 2008-04-08 12 -95 val_95 2008-04-08 12 -196 val_196 2008-04-08 12 -288 val_288 2008-04-08 12 -481 val_481 2008-04-08 12 -457 val_457 2008-04-08 12 -98 val_98 2008-04-08 12 -282 val_282 2008-04-08 12 -197 val_197 2008-04-08 12 -187 val_187 2008-04-08 12 -318 val_318 2008-04-08 12 -318 val_318 2008-04-08 12 -409 val_409 2008-04-08 12 -470 val_470 2008-04-08 12 -137 val_137 2008-04-08 12 -369 val_369 2008-04-08 12 -316 val_316 2008-04-08 12 -169 val_169 2008-04-08 12 -413 val_413 2008-04-08 12 -85 val_85 2008-04-08 12 -77 val_77 2008-04-08 12 -0 val_0 2008-04-08 12 -490 val_490 2008-04-08 12 -87 val_87 2008-04-08 12 -364 val_364 2008-04-08 12 -179 val_179 2008-04-08 12 -118 val_118 2008-04-08 12 -134 val_134 2008-04-08 12 -395 val_395 2008-04-08 12 -282 val_282 2008-04-08 12 -138 val_138 2008-04-08 12 -238 val_238 2008-04-08 12 -419 val_419 2008-04-08 12 -15 val_15 2008-04-08 12 -118 val_118 2008-04-08 12 -72 val_72 2008-04-08 12 -90 val_90 2008-04-08 12 -307 val_307 2008-04-08 12 -19 val_19 2008-04-08 12 -435 val_435 2008-04-08 12 -10 val_10 2008-04-08 12 -277 val_277 2008-04-08 12 -273 val_273 2008-04-08 12 -306 val_306 2008-04-08 12 -224 val_224 2008-04-08 12 -309 val_309 2008-04-08 12 -389 val_389 2008-04-08 12 -327 val_327 2008-04-08 12 -242 val_242 2008-04-08 12 -369 val_369 2008-04-08 12 -392 val_392 2008-04-08 12 -272 val_272 2008-04-08 12 -331 val_331 2008-04-08 12 -401 val_401 2008-04-08 12 -242 val_242 2008-04-08 12 -452 val_452 2008-04-08 12 -177 val_177 2008-04-08 12 -226 val_226 2008-04-08 12 -5 val_5 2008-04-08 12 -497 val_497 2008-04-08 12 -402 val_402 2008-04-08 12 -396 val_396 2008-04-08 12 -317 val_317 2008-04-08 12 -395 val_395 2008-04-08 12 -58 val_58 2008-04-08 12 -35 val_35 2008-04-08 12 -336 val_336 2008-04-08 12 -95 val_95 2008-04-08 12 -11 val_11 2008-04-08 12 -168 val_168 2008-04-08 12 -34 val_34 2008-04-08 12 -229 val_229 2008-04-08 12 -233 val_233 2008-04-08 12 -143 val_143 2008-04-08 12 -472 val_472 2008-04-08 12 -322 val_322 2008-04-08 12 -498 val_498 2008-04-08 12 -160 val_160 2008-04-08 12 -195 val_195 2008-04-08 12 -42 val_42 2008-04-08 12 -321 val_321 2008-04-08 12 -430 val_430 2008-04-08 12 -119 val_119 2008-04-08 12 -489 val_489 2008-04-08 12 -458 val_458 2008-04-08 12 -78 val_78 2008-04-08 12 -76 val_76 2008-04-08 12 -41 val_41 2008-04-08 12 -223 val_223 2008-04-08 12 -492 val_492 2008-04-08 12 -149 val_149 2008-04-08 12 -449 val_449 2008-04-08 12 -218 val_218 2008-04-08 12 -228 val_228 2008-04-08 12 -138 val_138 2008-04-08 12 -453 val_453 2008-04-08 12 -30 val_30 2008-04-08 12 -209 val_209 2008-04-08 12 -64 val_64 2008-04-08 12 -468 val_468 2008-04-08 12 -76 val_76 2008-04-08 12 -74 val_74 2008-04-08 12 -342 val_342 2008-04-08 12 -69 val_69 2008-04-08 12 -230 val_230 2008-04-08 12 -33 val_33 2008-04-08 12 -368 val_368 2008-04-08 12 -103 val_103 2008-04-08 12 -296 val_296 2008-04-08 12 -113 val_113 2008-04-08 12 -216 val_216 2008-04-08 12 -367 val_367 2008-04-08 12 -344 val_344 2008-04-08 12 -167 val_167 2008-04-08 12 -274 val_274 2008-04-08 12 -219 val_219 2008-04-08 12 -239 val_239 2008-04-08 12 -485 val_485 2008-04-08 12 -116 val_116 2008-04-08 12 -223 val_223 2008-04-08 12 -256 val_256 2008-04-08 12 -263 val_263 2008-04-08 12 -70 val_70 2008-04-08 12 -487 val_487 2008-04-08 12 -480 val_480 2008-04-08 12 -401 val_401 2008-04-08 12 -288 val_288 2008-04-08 12 -191 val_191 2008-04-08 12 -5 val_5 2008-04-08 12 -244 val_244 2008-04-08 12 -438 val_438 2008-04-08 12 -128 val_128 2008-04-08 12 -467 val_467 2008-04-08 12 -432 val_432 2008-04-08 12 -202 val_202 2008-04-08 12 -316 val_316 2008-04-08 12 -229 val_229 2008-04-08 12 -469 val_469 2008-04-08 12 -463 val_463 2008-04-08 12 -280 val_280 2008-04-08 12 -2 val_2 2008-04-08 12 -35 val_35 2008-04-08 12 -283 val_283 2008-04-08 12 -331 val_331 2008-04-08 12 -235 val_235 2008-04-08 12 -80 val_80 2008-04-08 12 -44 val_44 2008-04-08 12 -193 val_193 2008-04-08 12 -321 val_321 2008-04-08 12 -335 val_335 2008-04-08 12 -104 val_104 2008-04-08 12 -466 val_466 2008-04-08 12 -366 val_366 2008-04-08 12 -175 val_175 2008-04-08 12 -403 val_403 2008-04-08 12 -483 val_483 2008-04-08 12 -53 val_53 2008-04-08 12 -105 val_105 2008-04-08 12 -257 val_257 2008-04-08 12 -406 val_406 2008-04-08 12 -409 val_409 2008-04-08 12 -190 val_190 2008-04-08 12 -406 val_406 2008-04-08 12 -401 val_401 2008-04-08 12 -114 val_114 2008-04-08 12 -258 val_258 2008-04-08 12 -90 val_90 2008-04-08 12 -203 val_203 2008-04-08 12 -262 val_262 2008-04-08 12 -348 val_348 2008-04-08 12 -424 val_424 2008-04-08 12 -12 val_12 2008-04-08 12 -396 val_396 2008-04-08 12 -201 val_201 2008-04-08 12 -217 val_217 2008-04-08 12 -164 val_164 2008-04-08 12 -431 val_431 2008-04-08 12 -454 val_454 2008-04-08 12 -478 val_478 2008-04-08 12 -298 val_298 2008-04-08 12 -125 val_125 2008-04-08 12 -431 val_431 2008-04-08 12 -164 val_164 2008-04-08 12 -424 val_424 2008-04-08 12 -187 val_187 2008-04-08 12 -382 val_382 2008-04-08 12 -5 val_5 2008-04-08 12 -70 val_70 2008-04-08 12 -397 val_397 2008-04-08 12 -480 val_480 2008-04-08 12 -291 val_291 2008-04-08 12 -24 val_24 2008-04-08 12 -351 val_351 2008-04-08 12 -255 val_255 2008-04-08 12 -104 val_104 2008-04-08 12 -70 val_70 2008-04-08 12 -163 val_163 2008-04-08 12 -438 val_438 2008-04-08 12 -119 val_119 2008-04-08 12 -414 val_414 2008-04-08 12 -200 val_200 2008-04-08 12 -491 val_491 2008-04-08 12 -237 val_237 2008-04-08 12 -439 val_439 2008-04-08 12 -360 val_360 2008-04-08 12 -248 val_248 2008-04-08 12 -479 val_479 2008-04-08 12 -305 val_305 2008-04-08 12 -417 val_417 2008-04-08 12 -199 val_199 2008-04-08 12 -444 val_444 2008-04-08 12 -120 val_120 2008-04-08 12 -429 val_429 2008-04-08 12 -169 val_169 2008-04-08 12 -443 val_443 2008-04-08 12 -323 val_323 2008-04-08 12 -325 val_325 2008-04-08 12 -277 val_277 2008-04-08 12 -230 val_230 2008-04-08 12 -478 val_478 2008-04-08 12 -178 val_178 2008-04-08 12 -468 val_468 2008-04-08 12 -310 val_310 2008-04-08 12 -317 val_317 2008-04-08 12 -333 val_333 2008-04-08 12 -493 val_493 2008-04-08 12 -460 val_460 2008-04-08 12 -207 val_207 2008-04-08 12 -249 val_249 2008-04-08 12 -265 val_265 2008-04-08 12 -480 val_480 2008-04-08 12 -83 val_83 2008-04-08 12 -136 val_136 2008-04-08 12 -353 val_353 2008-04-08 12 -172 val_172 2008-04-08 12 -214 val_214 2008-04-08 12 -462 val_462 2008-04-08 12 -233 val_233 2008-04-08 12 -406 val_406 2008-04-08 12 -133 val_133 2008-04-08 12 -175 val_175 2008-04-08 12 -189 val_189 2008-04-08 12 -454 val_454 2008-04-08 12 -375 val_375 2008-04-08 12 -401 val_401 2008-04-08 12 -421 val_421 2008-04-08 12 -407 val_407 2008-04-08 12 -384 val_384 2008-04-08 12 -256 val_256 2008-04-08 12 -26 val_26 2008-04-08 12 -134 val_134 2008-04-08 12 -67 val_67 2008-04-08 12 -384 val_384 2008-04-08 12 -379 val_379 2008-04-08 12 -18 val_18 2008-04-08 12 -462 val_462 2008-04-08 12 -492 val_492 2008-04-08 12 -100 val_100 2008-04-08 12 -298 val_298 2008-04-08 12 -9 val_9 2008-04-08 12 -341 val_341 2008-04-08 12 -498 val_498 2008-04-08 12 -146 val_146 2008-04-08 12 -458 val_458 2008-04-08 12 -362 val_362 2008-04-08 12 -186 val_186 2008-04-08 12 -285 val_285 2008-04-08 12 -348 val_348 2008-04-08 12 -167 val_167 2008-04-08 12 -18 val_18 2008-04-08 12 -273 val_273 2008-04-08 12 -183 val_183 2008-04-08 12 -281 val_281 2008-04-08 12 -344 val_344 2008-04-08 12 -97 val_97 2008-04-08 12 -469 val_469 2008-04-08 12 -315 val_315 2008-04-08 12 -84 val_84 2008-04-08 12 -28 val_28 2008-04-08 12 -37 val_37 2008-04-08 12 -448 val_448 2008-04-08 12 -152 val_152 2008-04-08 12 -348 val_348 2008-04-08 12 -307 val_307 2008-04-08 12 -194 val_194 2008-04-08 12 -414 val_414 2008-04-08 12 -477 val_477 2008-04-08 12 -222 val_222 2008-04-08 12 -126 val_126 2008-04-08 12 -90 val_90 2008-04-08 12 -169 val_169 2008-04-08 12 -403 val_403 2008-04-08 12 -400 val_400 2008-04-08 12 -200 val_200 2008-04-08 12 -97 val_97 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 b/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 deleted file mode 100644 index 653516475da22..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 +++ /dev/null @@ -1,1000 +0,0 @@ -238 val_238 2008-12-31 11 -86 val_86 2008-12-31 11 -311 val_311 2008-12-31 11 -27 val_27 2008-12-31 11 -165 val_165 2008-12-31 11 -409 val_409 2008-12-31 11 -255 val_255 2008-12-31 11 -278 val_278 2008-12-31 11 -98 val_98 2008-12-31 11 -484 val_484 2008-12-31 11 -265 val_265 2008-12-31 11 -193 val_193 2008-12-31 11 -401 val_401 2008-12-31 11 -150 val_150 2008-12-31 11 -273 val_273 2008-12-31 11 -224 val_224 2008-12-31 11 -369 val_369 2008-12-31 11 -66 val_66 2008-12-31 11 -128 val_128 2008-12-31 11 -213 val_213 2008-12-31 11 -146 val_146 2008-12-31 11 -406 val_406 2008-12-31 11 -429 val_429 2008-12-31 11 -374 val_374 2008-12-31 11 -152 val_152 2008-12-31 11 -469 val_469 2008-12-31 11 -145 val_145 2008-12-31 11 -495 val_495 2008-12-31 11 -37 val_37 2008-12-31 11 -327 val_327 2008-12-31 11 -281 val_281 2008-12-31 11 -277 val_277 2008-12-31 11 -209 val_209 2008-12-31 11 -15 val_15 2008-12-31 11 -82 val_82 2008-12-31 11 -403 val_403 2008-12-31 11 -166 val_166 2008-12-31 11 -417 val_417 2008-12-31 11 -430 val_430 2008-12-31 11 -252 val_252 2008-12-31 11 -292 val_292 2008-12-31 11 -219 val_219 2008-12-31 11 -287 val_287 2008-12-31 11 -153 val_153 2008-12-31 11 -193 val_193 2008-12-31 11 -338 val_338 2008-12-31 11 -446 val_446 2008-12-31 11 -459 val_459 2008-12-31 11 -394 val_394 2008-12-31 11 -237 val_237 2008-12-31 11 -482 val_482 2008-12-31 11 -174 val_174 2008-12-31 11 -413 val_413 2008-12-31 11 -494 val_494 2008-12-31 11 -207 val_207 2008-12-31 11 -199 val_199 2008-12-31 11 -466 val_466 2008-12-31 11 -208 val_208 2008-12-31 11 -174 val_174 2008-12-31 11 -399 val_399 2008-12-31 11 -396 val_396 2008-12-31 11 -247 val_247 2008-12-31 11 -417 val_417 2008-12-31 11 -489 val_489 2008-12-31 11 -162 val_162 2008-12-31 11 -377 val_377 2008-12-31 11 -397 val_397 2008-12-31 11 -309 val_309 2008-12-31 11 -365 val_365 2008-12-31 11 -266 val_266 2008-12-31 11 -439 val_439 2008-12-31 11 -342 val_342 2008-12-31 11 -367 val_367 2008-12-31 11 -325 val_325 2008-12-31 11 -167 val_167 2008-12-31 11 -195 val_195 2008-12-31 11 -475 val_475 2008-12-31 11 -17 val_17 2008-12-31 11 -113 val_113 2008-12-31 11 -155 val_155 2008-12-31 11 -203 val_203 2008-12-31 11 -339 val_339 2008-12-31 11 -0 val_0 2008-12-31 11 -455 val_455 2008-12-31 11 -128 val_128 2008-12-31 11 -311 val_311 2008-12-31 11 -316 val_316 2008-12-31 11 -57 val_57 2008-12-31 11 -302 val_302 2008-12-31 11 -205 val_205 2008-12-31 11 -149 val_149 2008-12-31 11 -438 val_438 2008-12-31 11 -345 val_345 2008-12-31 11 -129 val_129 2008-12-31 11 -170 val_170 2008-12-31 11 -20 val_20 2008-12-31 11 -489 val_489 2008-12-31 11 -157 val_157 2008-12-31 11 -378 val_378 2008-12-31 11 -221 val_221 2008-12-31 11 -92 val_92 2008-12-31 11 -111 val_111 2008-12-31 11 -47 val_47 2008-12-31 11 -72 val_72 2008-12-31 11 -4 val_4 2008-12-31 11 -280 val_280 2008-12-31 11 -35 val_35 2008-12-31 11 -427 val_427 2008-12-31 11 -277 val_277 2008-12-31 11 -208 val_208 2008-12-31 11 -356 val_356 2008-12-31 11 -399 val_399 2008-12-31 11 -169 val_169 2008-12-31 11 -382 val_382 2008-12-31 11 -498 val_498 2008-12-31 11 -125 val_125 2008-12-31 11 -386 val_386 2008-12-31 11 -437 val_437 2008-12-31 11 -469 val_469 2008-12-31 11 -192 val_192 2008-12-31 11 -286 val_286 2008-12-31 11 -187 val_187 2008-12-31 11 -176 val_176 2008-12-31 11 -54 val_54 2008-12-31 11 -459 val_459 2008-12-31 11 -51 val_51 2008-12-31 11 -138 val_138 2008-12-31 11 -103 val_103 2008-12-31 11 -239 val_239 2008-12-31 11 -213 val_213 2008-12-31 11 -216 val_216 2008-12-31 11 -430 val_430 2008-12-31 11 -278 val_278 2008-12-31 11 -176 val_176 2008-12-31 11 -289 val_289 2008-12-31 11 -221 val_221 2008-12-31 11 -65 val_65 2008-12-31 11 -318 val_318 2008-12-31 11 -332 val_332 2008-12-31 11 -311 val_311 2008-12-31 11 -275 val_275 2008-12-31 11 -137 val_137 2008-12-31 11 -241 val_241 2008-12-31 11 -83 val_83 2008-12-31 11 -333 val_333 2008-12-31 11 -180 val_180 2008-12-31 11 -284 val_284 2008-12-31 11 -12 val_12 2008-12-31 11 -230 val_230 2008-12-31 11 -181 val_181 2008-12-31 11 -67 val_67 2008-12-31 11 -260 val_260 2008-12-31 11 -404 val_404 2008-12-31 11 -384 val_384 2008-12-31 11 -489 val_489 2008-12-31 11 -353 val_353 2008-12-31 11 -373 val_373 2008-12-31 11 -272 val_272 2008-12-31 11 -138 val_138 2008-12-31 11 -217 val_217 2008-12-31 11 -84 val_84 2008-12-31 11 -348 val_348 2008-12-31 11 -466 val_466 2008-12-31 11 -58 val_58 2008-12-31 11 -8 val_8 2008-12-31 11 -411 val_411 2008-12-31 11 -230 val_230 2008-12-31 11 -208 val_208 2008-12-31 11 -348 val_348 2008-12-31 11 -24 val_24 2008-12-31 11 -463 val_463 2008-12-31 11 -431 val_431 2008-12-31 11 -179 val_179 2008-12-31 11 -172 val_172 2008-12-31 11 -42 val_42 2008-12-31 11 -129 val_129 2008-12-31 11 -158 val_158 2008-12-31 11 -119 val_119 2008-12-31 11 -496 val_496 2008-12-31 11 -0 val_0 2008-12-31 11 -322 val_322 2008-12-31 11 -197 val_197 2008-12-31 11 -468 val_468 2008-12-31 11 -393 val_393 2008-12-31 11 -454 val_454 2008-12-31 11 -100 val_100 2008-12-31 11 -298 val_298 2008-12-31 11 -199 val_199 2008-12-31 11 -191 val_191 2008-12-31 11 -418 val_418 2008-12-31 11 -96 val_96 2008-12-31 11 -26 val_26 2008-12-31 11 -165 val_165 2008-12-31 11 -327 val_327 2008-12-31 11 -230 val_230 2008-12-31 11 -205 val_205 2008-12-31 11 -120 val_120 2008-12-31 11 -131 val_131 2008-12-31 11 -51 val_51 2008-12-31 11 -404 val_404 2008-12-31 11 -43 val_43 2008-12-31 11 -436 val_436 2008-12-31 11 -156 val_156 2008-12-31 11 -469 val_469 2008-12-31 11 -468 val_468 2008-12-31 11 -308 val_308 2008-12-31 11 -95 val_95 2008-12-31 11 -196 val_196 2008-12-31 11 -288 val_288 2008-12-31 11 -481 val_481 2008-12-31 11 -457 val_457 2008-12-31 11 -98 val_98 2008-12-31 11 -282 val_282 2008-12-31 11 -197 val_197 2008-12-31 11 -187 val_187 2008-12-31 11 -318 val_318 2008-12-31 11 -318 val_318 2008-12-31 11 -409 val_409 2008-12-31 11 -470 val_470 2008-12-31 11 -137 val_137 2008-12-31 11 -369 val_369 2008-12-31 11 -316 val_316 2008-12-31 11 -169 val_169 2008-12-31 11 -413 val_413 2008-12-31 11 -85 val_85 2008-12-31 11 -77 val_77 2008-12-31 11 -0 val_0 2008-12-31 11 -490 val_490 2008-12-31 11 -87 val_87 2008-12-31 11 -364 val_364 2008-12-31 11 -179 val_179 2008-12-31 11 -118 val_118 2008-12-31 11 -134 val_134 2008-12-31 11 -395 val_395 2008-12-31 11 -282 val_282 2008-12-31 11 -138 val_138 2008-12-31 11 -238 val_238 2008-12-31 11 -419 val_419 2008-12-31 11 -15 val_15 2008-12-31 11 -118 val_118 2008-12-31 11 -72 val_72 2008-12-31 11 -90 val_90 2008-12-31 11 -307 val_307 2008-12-31 11 -19 val_19 2008-12-31 11 -435 val_435 2008-12-31 11 -10 val_10 2008-12-31 11 -277 val_277 2008-12-31 11 -273 val_273 2008-12-31 11 -306 val_306 2008-12-31 11 -224 val_224 2008-12-31 11 -309 val_309 2008-12-31 11 -389 val_389 2008-12-31 11 -327 val_327 2008-12-31 11 -242 val_242 2008-12-31 11 -369 val_369 2008-12-31 11 -392 val_392 2008-12-31 11 -272 val_272 2008-12-31 11 -331 val_331 2008-12-31 11 -401 val_401 2008-12-31 11 -242 val_242 2008-12-31 11 -452 val_452 2008-12-31 11 -177 val_177 2008-12-31 11 -226 val_226 2008-12-31 11 -5 val_5 2008-12-31 11 -497 val_497 2008-12-31 11 -402 val_402 2008-12-31 11 -396 val_396 2008-12-31 11 -317 val_317 2008-12-31 11 -395 val_395 2008-12-31 11 -58 val_58 2008-12-31 11 -35 val_35 2008-12-31 11 -336 val_336 2008-12-31 11 -95 val_95 2008-12-31 11 -11 val_11 2008-12-31 11 -168 val_168 2008-12-31 11 -34 val_34 2008-12-31 11 -229 val_229 2008-12-31 11 -233 val_233 2008-12-31 11 -143 val_143 2008-12-31 11 -472 val_472 2008-12-31 11 -322 val_322 2008-12-31 11 -498 val_498 2008-12-31 11 -160 val_160 2008-12-31 11 -195 val_195 2008-12-31 11 -42 val_42 2008-12-31 11 -321 val_321 2008-12-31 11 -430 val_430 2008-12-31 11 -119 val_119 2008-12-31 11 -489 val_489 2008-12-31 11 -458 val_458 2008-12-31 11 -78 val_78 2008-12-31 11 -76 val_76 2008-12-31 11 -41 val_41 2008-12-31 11 -223 val_223 2008-12-31 11 -492 val_492 2008-12-31 11 -149 val_149 2008-12-31 11 -449 val_449 2008-12-31 11 -218 val_218 2008-12-31 11 -228 val_228 2008-12-31 11 -138 val_138 2008-12-31 11 -453 val_453 2008-12-31 11 -30 val_30 2008-12-31 11 -209 val_209 2008-12-31 11 -64 val_64 2008-12-31 11 -468 val_468 2008-12-31 11 -76 val_76 2008-12-31 11 -74 val_74 2008-12-31 11 -342 val_342 2008-12-31 11 -69 val_69 2008-12-31 11 -230 val_230 2008-12-31 11 -33 val_33 2008-12-31 11 -368 val_368 2008-12-31 11 -103 val_103 2008-12-31 11 -296 val_296 2008-12-31 11 -113 val_113 2008-12-31 11 -216 val_216 2008-12-31 11 -367 val_367 2008-12-31 11 -344 val_344 2008-12-31 11 -167 val_167 2008-12-31 11 -274 val_274 2008-12-31 11 -219 val_219 2008-12-31 11 -239 val_239 2008-12-31 11 -485 val_485 2008-12-31 11 -116 val_116 2008-12-31 11 -223 val_223 2008-12-31 11 -256 val_256 2008-12-31 11 -263 val_263 2008-12-31 11 -70 val_70 2008-12-31 11 -487 val_487 2008-12-31 11 -480 val_480 2008-12-31 11 -401 val_401 2008-12-31 11 -288 val_288 2008-12-31 11 -191 val_191 2008-12-31 11 -5 val_5 2008-12-31 11 -244 val_244 2008-12-31 11 -438 val_438 2008-12-31 11 -128 val_128 2008-12-31 11 -467 val_467 2008-12-31 11 -432 val_432 2008-12-31 11 -202 val_202 2008-12-31 11 -316 val_316 2008-12-31 11 -229 val_229 2008-12-31 11 -469 val_469 2008-12-31 11 -463 val_463 2008-12-31 11 -280 val_280 2008-12-31 11 -2 val_2 2008-12-31 11 -35 val_35 2008-12-31 11 -283 val_283 2008-12-31 11 -331 val_331 2008-12-31 11 -235 val_235 2008-12-31 11 -80 val_80 2008-12-31 11 -44 val_44 2008-12-31 11 -193 val_193 2008-12-31 11 -321 val_321 2008-12-31 11 -335 val_335 2008-12-31 11 -104 val_104 2008-12-31 11 -466 val_466 2008-12-31 11 -366 val_366 2008-12-31 11 -175 val_175 2008-12-31 11 -403 val_403 2008-12-31 11 -483 val_483 2008-12-31 11 -53 val_53 2008-12-31 11 -105 val_105 2008-12-31 11 -257 val_257 2008-12-31 11 -406 val_406 2008-12-31 11 -409 val_409 2008-12-31 11 -190 val_190 2008-12-31 11 -406 val_406 2008-12-31 11 -401 val_401 2008-12-31 11 -114 val_114 2008-12-31 11 -258 val_258 2008-12-31 11 -90 val_90 2008-12-31 11 -203 val_203 2008-12-31 11 -262 val_262 2008-12-31 11 -348 val_348 2008-12-31 11 -424 val_424 2008-12-31 11 -12 val_12 2008-12-31 11 -396 val_396 2008-12-31 11 -201 val_201 2008-12-31 11 -217 val_217 2008-12-31 11 -164 val_164 2008-12-31 11 -431 val_431 2008-12-31 11 -454 val_454 2008-12-31 11 -478 val_478 2008-12-31 11 -298 val_298 2008-12-31 11 -125 val_125 2008-12-31 11 -431 val_431 2008-12-31 11 -164 val_164 2008-12-31 11 -424 val_424 2008-12-31 11 -187 val_187 2008-12-31 11 -382 val_382 2008-12-31 11 -5 val_5 2008-12-31 11 -70 val_70 2008-12-31 11 -397 val_397 2008-12-31 11 -480 val_480 2008-12-31 11 -291 val_291 2008-12-31 11 -24 val_24 2008-12-31 11 -351 val_351 2008-12-31 11 -255 val_255 2008-12-31 11 -104 val_104 2008-12-31 11 -70 val_70 2008-12-31 11 -163 val_163 2008-12-31 11 -438 val_438 2008-12-31 11 -119 val_119 2008-12-31 11 -414 val_414 2008-12-31 11 -200 val_200 2008-12-31 11 -491 val_491 2008-12-31 11 -237 val_237 2008-12-31 11 -439 val_439 2008-12-31 11 -360 val_360 2008-12-31 11 -248 val_248 2008-12-31 11 -479 val_479 2008-12-31 11 -305 val_305 2008-12-31 11 -417 val_417 2008-12-31 11 -199 val_199 2008-12-31 11 -444 val_444 2008-12-31 11 -120 val_120 2008-12-31 11 -429 val_429 2008-12-31 11 -169 val_169 2008-12-31 11 -443 val_443 2008-12-31 11 -323 val_323 2008-12-31 11 -325 val_325 2008-12-31 11 -277 val_277 2008-12-31 11 -230 val_230 2008-12-31 11 -478 val_478 2008-12-31 11 -178 val_178 2008-12-31 11 -468 val_468 2008-12-31 11 -310 val_310 2008-12-31 11 -317 val_317 2008-12-31 11 -333 val_333 2008-12-31 11 -493 val_493 2008-12-31 11 -460 val_460 2008-12-31 11 -207 val_207 2008-12-31 11 -249 val_249 2008-12-31 11 -265 val_265 2008-12-31 11 -480 val_480 2008-12-31 11 -83 val_83 2008-12-31 11 -136 val_136 2008-12-31 11 -353 val_353 2008-12-31 11 -172 val_172 2008-12-31 11 -214 val_214 2008-12-31 11 -462 val_462 2008-12-31 11 -233 val_233 2008-12-31 11 -406 val_406 2008-12-31 11 -133 val_133 2008-12-31 11 -175 val_175 2008-12-31 11 -189 val_189 2008-12-31 11 -454 val_454 2008-12-31 11 -375 val_375 2008-12-31 11 -401 val_401 2008-12-31 11 -421 val_421 2008-12-31 11 -407 val_407 2008-12-31 11 -384 val_384 2008-12-31 11 -256 val_256 2008-12-31 11 -26 val_26 2008-12-31 11 -134 val_134 2008-12-31 11 -67 val_67 2008-12-31 11 -384 val_384 2008-12-31 11 -379 val_379 2008-12-31 11 -18 val_18 2008-12-31 11 -462 val_462 2008-12-31 11 -492 val_492 2008-12-31 11 -100 val_100 2008-12-31 11 -298 val_298 2008-12-31 11 -9 val_9 2008-12-31 11 -341 val_341 2008-12-31 11 -498 val_498 2008-12-31 11 -146 val_146 2008-12-31 11 -458 val_458 2008-12-31 11 -362 val_362 2008-12-31 11 -186 val_186 2008-12-31 11 -285 val_285 2008-12-31 11 -348 val_348 2008-12-31 11 -167 val_167 2008-12-31 11 -18 val_18 2008-12-31 11 -273 val_273 2008-12-31 11 -183 val_183 2008-12-31 11 -281 val_281 2008-12-31 11 -344 val_344 2008-12-31 11 -97 val_97 2008-12-31 11 -469 val_469 2008-12-31 11 -315 val_315 2008-12-31 11 -84 val_84 2008-12-31 11 -28 val_28 2008-12-31 11 -37 val_37 2008-12-31 11 -448 val_448 2008-12-31 11 -152 val_152 2008-12-31 11 -348 val_348 2008-12-31 11 -307 val_307 2008-12-31 11 -194 val_194 2008-12-31 11 -414 val_414 2008-12-31 11 -477 val_477 2008-12-31 11 -222 val_222 2008-12-31 11 -126 val_126 2008-12-31 11 -90 val_90 2008-12-31 11 -169 val_169 2008-12-31 11 -403 val_403 2008-12-31 11 -400 val_400 2008-12-31 11 -200 val_200 2008-12-31 11 -97 val_97 2008-12-31 11 -238 val_238 2008-12-31 12 -86 val_86 2008-12-31 12 -311 val_311 2008-12-31 12 -27 val_27 2008-12-31 12 -165 val_165 2008-12-31 12 -409 val_409 2008-12-31 12 -255 val_255 2008-12-31 12 -278 val_278 2008-12-31 12 -98 val_98 2008-12-31 12 -484 val_484 2008-12-31 12 -265 val_265 2008-12-31 12 -193 val_193 2008-12-31 12 -401 val_401 2008-12-31 12 -150 val_150 2008-12-31 12 -273 val_273 2008-12-31 12 -224 val_224 2008-12-31 12 -369 val_369 2008-12-31 12 -66 val_66 2008-12-31 12 -128 val_128 2008-12-31 12 -213 val_213 2008-12-31 12 -146 val_146 2008-12-31 12 -406 val_406 2008-12-31 12 -429 val_429 2008-12-31 12 -374 val_374 2008-12-31 12 -152 val_152 2008-12-31 12 -469 val_469 2008-12-31 12 -145 val_145 2008-12-31 12 -495 val_495 2008-12-31 12 -37 val_37 2008-12-31 12 -327 val_327 2008-12-31 12 -281 val_281 2008-12-31 12 -277 val_277 2008-12-31 12 -209 val_209 2008-12-31 12 -15 val_15 2008-12-31 12 -82 val_82 2008-12-31 12 -403 val_403 2008-12-31 12 -166 val_166 2008-12-31 12 -417 val_417 2008-12-31 12 -430 val_430 2008-12-31 12 -252 val_252 2008-12-31 12 -292 val_292 2008-12-31 12 -219 val_219 2008-12-31 12 -287 val_287 2008-12-31 12 -153 val_153 2008-12-31 12 -193 val_193 2008-12-31 12 -338 val_338 2008-12-31 12 -446 val_446 2008-12-31 12 -459 val_459 2008-12-31 12 -394 val_394 2008-12-31 12 -237 val_237 2008-12-31 12 -482 val_482 2008-12-31 12 -174 val_174 2008-12-31 12 -413 val_413 2008-12-31 12 -494 val_494 2008-12-31 12 -207 val_207 2008-12-31 12 -199 val_199 2008-12-31 12 -466 val_466 2008-12-31 12 -208 val_208 2008-12-31 12 -174 val_174 2008-12-31 12 -399 val_399 2008-12-31 12 -396 val_396 2008-12-31 12 -247 val_247 2008-12-31 12 -417 val_417 2008-12-31 12 -489 val_489 2008-12-31 12 -162 val_162 2008-12-31 12 -377 val_377 2008-12-31 12 -397 val_397 2008-12-31 12 -309 val_309 2008-12-31 12 -365 val_365 2008-12-31 12 -266 val_266 2008-12-31 12 -439 val_439 2008-12-31 12 -342 val_342 2008-12-31 12 -367 val_367 2008-12-31 12 -325 val_325 2008-12-31 12 -167 val_167 2008-12-31 12 -195 val_195 2008-12-31 12 -475 val_475 2008-12-31 12 -17 val_17 2008-12-31 12 -113 val_113 2008-12-31 12 -155 val_155 2008-12-31 12 -203 val_203 2008-12-31 12 -339 val_339 2008-12-31 12 -0 val_0 2008-12-31 12 -455 val_455 2008-12-31 12 -128 val_128 2008-12-31 12 -311 val_311 2008-12-31 12 -316 val_316 2008-12-31 12 -57 val_57 2008-12-31 12 -302 val_302 2008-12-31 12 -205 val_205 2008-12-31 12 -149 val_149 2008-12-31 12 -438 val_438 2008-12-31 12 -345 val_345 2008-12-31 12 -129 val_129 2008-12-31 12 -170 val_170 2008-12-31 12 -20 val_20 2008-12-31 12 -489 val_489 2008-12-31 12 -157 val_157 2008-12-31 12 -378 val_378 2008-12-31 12 -221 val_221 2008-12-31 12 -92 val_92 2008-12-31 12 -111 val_111 2008-12-31 12 -47 val_47 2008-12-31 12 -72 val_72 2008-12-31 12 -4 val_4 2008-12-31 12 -280 val_280 2008-12-31 12 -35 val_35 2008-12-31 12 -427 val_427 2008-12-31 12 -277 val_277 2008-12-31 12 -208 val_208 2008-12-31 12 -356 val_356 2008-12-31 12 -399 val_399 2008-12-31 12 -169 val_169 2008-12-31 12 -382 val_382 2008-12-31 12 -498 val_498 2008-12-31 12 -125 val_125 2008-12-31 12 -386 val_386 2008-12-31 12 -437 val_437 2008-12-31 12 -469 val_469 2008-12-31 12 -192 val_192 2008-12-31 12 -286 val_286 2008-12-31 12 -187 val_187 2008-12-31 12 -176 val_176 2008-12-31 12 -54 val_54 2008-12-31 12 -459 val_459 2008-12-31 12 -51 val_51 2008-12-31 12 -138 val_138 2008-12-31 12 -103 val_103 2008-12-31 12 -239 val_239 2008-12-31 12 -213 val_213 2008-12-31 12 -216 val_216 2008-12-31 12 -430 val_430 2008-12-31 12 -278 val_278 2008-12-31 12 -176 val_176 2008-12-31 12 -289 val_289 2008-12-31 12 -221 val_221 2008-12-31 12 -65 val_65 2008-12-31 12 -318 val_318 2008-12-31 12 -332 val_332 2008-12-31 12 -311 val_311 2008-12-31 12 -275 val_275 2008-12-31 12 -137 val_137 2008-12-31 12 -241 val_241 2008-12-31 12 -83 val_83 2008-12-31 12 -333 val_333 2008-12-31 12 -180 val_180 2008-12-31 12 -284 val_284 2008-12-31 12 -12 val_12 2008-12-31 12 -230 val_230 2008-12-31 12 -181 val_181 2008-12-31 12 -67 val_67 2008-12-31 12 -260 val_260 2008-12-31 12 -404 val_404 2008-12-31 12 -384 val_384 2008-12-31 12 -489 val_489 2008-12-31 12 -353 val_353 2008-12-31 12 -373 val_373 2008-12-31 12 -272 val_272 2008-12-31 12 -138 val_138 2008-12-31 12 -217 val_217 2008-12-31 12 -84 val_84 2008-12-31 12 -348 val_348 2008-12-31 12 -466 val_466 2008-12-31 12 -58 val_58 2008-12-31 12 -8 val_8 2008-12-31 12 -411 val_411 2008-12-31 12 -230 val_230 2008-12-31 12 -208 val_208 2008-12-31 12 -348 val_348 2008-12-31 12 -24 val_24 2008-12-31 12 -463 val_463 2008-12-31 12 -431 val_431 2008-12-31 12 -179 val_179 2008-12-31 12 -172 val_172 2008-12-31 12 -42 val_42 2008-12-31 12 -129 val_129 2008-12-31 12 -158 val_158 2008-12-31 12 -119 val_119 2008-12-31 12 -496 val_496 2008-12-31 12 -0 val_0 2008-12-31 12 -322 val_322 2008-12-31 12 -197 val_197 2008-12-31 12 -468 val_468 2008-12-31 12 -393 val_393 2008-12-31 12 -454 val_454 2008-12-31 12 -100 val_100 2008-12-31 12 -298 val_298 2008-12-31 12 -199 val_199 2008-12-31 12 -191 val_191 2008-12-31 12 -418 val_418 2008-12-31 12 -96 val_96 2008-12-31 12 -26 val_26 2008-12-31 12 -165 val_165 2008-12-31 12 -327 val_327 2008-12-31 12 -230 val_230 2008-12-31 12 -205 val_205 2008-12-31 12 -120 val_120 2008-12-31 12 -131 val_131 2008-12-31 12 -51 val_51 2008-12-31 12 -404 val_404 2008-12-31 12 -43 val_43 2008-12-31 12 -436 val_436 2008-12-31 12 -156 val_156 2008-12-31 12 -469 val_469 2008-12-31 12 -468 val_468 2008-12-31 12 -308 val_308 2008-12-31 12 -95 val_95 2008-12-31 12 -196 val_196 2008-12-31 12 -288 val_288 2008-12-31 12 -481 val_481 2008-12-31 12 -457 val_457 2008-12-31 12 -98 val_98 2008-12-31 12 -282 val_282 2008-12-31 12 -197 val_197 2008-12-31 12 -187 val_187 2008-12-31 12 -318 val_318 2008-12-31 12 -318 val_318 2008-12-31 12 -409 val_409 2008-12-31 12 -470 val_470 2008-12-31 12 -137 val_137 2008-12-31 12 -369 val_369 2008-12-31 12 -316 val_316 2008-12-31 12 -169 val_169 2008-12-31 12 -413 val_413 2008-12-31 12 -85 val_85 2008-12-31 12 -77 val_77 2008-12-31 12 -0 val_0 2008-12-31 12 -490 val_490 2008-12-31 12 -87 val_87 2008-12-31 12 -364 val_364 2008-12-31 12 -179 val_179 2008-12-31 12 -118 val_118 2008-12-31 12 -134 val_134 2008-12-31 12 -395 val_395 2008-12-31 12 -282 val_282 2008-12-31 12 -138 val_138 2008-12-31 12 -238 val_238 2008-12-31 12 -419 val_419 2008-12-31 12 -15 val_15 2008-12-31 12 -118 val_118 2008-12-31 12 -72 val_72 2008-12-31 12 -90 val_90 2008-12-31 12 -307 val_307 2008-12-31 12 -19 val_19 2008-12-31 12 -435 val_435 2008-12-31 12 -10 val_10 2008-12-31 12 -277 val_277 2008-12-31 12 -273 val_273 2008-12-31 12 -306 val_306 2008-12-31 12 -224 val_224 2008-12-31 12 -309 val_309 2008-12-31 12 -389 val_389 2008-12-31 12 -327 val_327 2008-12-31 12 -242 val_242 2008-12-31 12 -369 val_369 2008-12-31 12 -392 val_392 2008-12-31 12 -272 val_272 2008-12-31 12 -331 val_331 2008-12-31 12 -401 val_401 2008-12-31 12 -242 val_242 2008-12-31 12 -452 val_452 2008-12-31 12 -177 val_177 2008-12-31 12 -226 val_226 2008-12-31 12 -5 val_5 2008-12-31 12 -497 val_497 2008-12-31 12 -402 val_402 2008-12-31 12 -396 val_396 2008-12-31 12 -317 val_317 2008-12-31 12 -395 val_395 2008-12-31 12 -58 val_58 2008-12-31 12 -35 val_35 2008-12-31 12 -336 val_336 2008-12-31 12 -95 val_95 2008-12-31 12 -11 val_11 2008-12-31 12 -168 val_168 2008-12-31 12 -34 val_34 2008-12-31 12 -229 val_229 2008-12-31 12 -233 val_233 2008-12-31 12 -143 val_143 2008-12-31 12 -472 val_472 2008-12-31 12 -322 val_322 2008-12-31 12 -498 val_498 2008-12-31 12 -160 val_160 2008-12-31 12 -195 val_195 2008-12-31 12 -42 val_42 2008-12-31 12 -321 val_321 2008-12-31 12 -430 val_430 2008-12-31 12 -119 val_119 2008-12-31 12 -489 val_489 2008-12-31 12 -458 val_458 2008-12-31 12 -78 val_78 2008-12-31 12 -76 val_76 2008-12-31 12 -41 val_41 2008-12-31 12 -223 val_223 2008-12-31 12 -492 val_492 2008-12-31 12 -149 val_149 2008-12-31 12 -449 val_449 2008-12-31 12 -218 val_218 2008-12-31 12 -228 val_228 2008-12-31 12 -138 val_138 2008-12-31 12 -453 val_453 2008-12-31 12 -30 val_30 2008-12-31 12 -209 val_209 2008-12-31 12 -64 val_64 2008-12-31 12 -468 val_468 2008-12-31 12 -76 val_76 2008-12-31 12 -74 val_74 2008-12-31 12 -342 val_342 2008-12-31 12 -69 val_69 2008-12-31 12 -230 val_230 2008-12-31 12 -33 val_33 2008-12-31 12 -368 val_368 2008-12-31 12 -103 val_103 2008-12-31 12 -296 val_296 2008-12-31 12 -113 val_113 2008-12-31 12 -216 val_216 2008-12-31 12 -367 val_367 2008-12-31 12 -344 val_344 2008-12-31 12 -167 val_167 2008-12-31 12 -274 val_274 2008-12-31 12 -219 val_219 2008-12-31 12 -239 val_239 2008-12-31 12 -485 val_485 2008-12-31 12 -116 val_116 2008-12-31 12 -223 val_223 2008-12-31 12 -256 val_256 2008-12-31 12 -263 val_263 2008-12-31 12 -70 val_70 2008-12-31 12 -487 val_487 2008-12-31 12 -480 val_480 2008-12-31 12 -401 val_401 2008-12-31 12 -288 val_288 2008-12-31 12 -191 val_191 2008-12-31 12 -5 val_5 2008-12-31 12 -244 val_244 2008-12-31 12 -438 val_438 2008-12-31 12 -128 val_128 2008-12-31 12 -467 val_467 2008-12-31 12 -432 val_432 2008-12-31 12 -202 val_202 2008-12-31 12 -316 val_316 2008-12-31 12 -229 val_229 2008-12-31 12 -469 val_469 2008-12-31 12 -463 val_463 2008-12-31 12 -280 val_280 2008-12-31 12 -2 val_2 2008-12-31 12 -35 val_35 2008-12-31 12 -283 val_283 2008-12-31 12 -331 val_331 2008-12-31 12 -235 val_235 2008-12-31 12 -80 val_80 2008-12-31 12 -44 val_44 2008-12-31 12 -193 val_193 2008-12-31 12 -321 val_321 2008-12-31 12 -335 val_335 2008-12-31 12 -104 val_104 2008-12-31 12 -466 val_466 2008-12-31 12 -366 val_366 2008-12-31 12 -175 val_175 2008-12-31 12 -403 val_403 2008-12-31 12 -483 val_483 2008-12-31 12 -53 val_53 2008-12-31 12 -105 val_105 2008-12-31 12 -257 val_257 2008-12-31 12 -406 val_406 2008-12-31 12 -409 val_409 2008-12-31 12 -190 val_190 2008-12-31 12 -406 val_406 2008-12-31 12 -401 val_401 2008-12-31 12 -114 val_114 2008-12-31 12 -258 val_258 2008-12-31 12 -90 val_90 2008-12-31 12 -203 val_203 2008-12-31 12 -262 val_262 2008-12-31 12 -348 val_348 2008-12-31 12 -424 val_424 2008-12-31 12 -12 val_12 2008-12-31 12 -396 val_396 2008-12-31 12 -201 val_201 2008-12-31 12 -217 val_217 2008-12-31 12 -164 val_164 2008-12-31 12 -431 val_431 2008-12-31 12 -454 val_454 2008-12-31 12 -478 val_478 2008-12-31 12 -298 val_298 2008-12-31 12 -125 val_125 2008-12-31 12 -431 val_431 2008-12-31 12 -164 val_164 2008-12-31 12 -424 val_424 2008-12-31 12 -187 val_187 2008-12-31 12 -382 val_382 2008-12-31 12 -5 val_5 2008-12-31 12 -70 val_70 2008-12-31 12 -397 val_397 2008-12-31 12 -480 val_480 2008-12-31 12 -291 val_291 2008-12-31 12 -24 val_24 2008-12-31 12 -351 val_351 2008-12-31 12 -255 val_255 2008-12-31 12 -104 val_104 2008-12-31 12 -70 val_70 2008-12-31 12 -163 val_163 2008-12-31 12 -438 val_438 2008-12-31 12 -119 val_119 2008-12-31 12 -414 val_414 2008-12-31 12 -200 val_200 2008-12-31 12 -491 val_491 2008-12-31 12 -237 val_237 2008-12-31 12 -439 val_439 2008-12-31 12 -360 val_360 2008-12-31 12 -248 val_248 2008-12-31 12 -479 val_479 2008-12-31 12 -305 val_305 2008-12-31 12 -417 val_417 2008-12-31 12 -199 val_199 2008-12-31 12 -444 val_444 2008-12-31 12 -120 val_120 2008-12-31 12 -429 val_429 2008-12-31 12 -169 val_169 2008-12-31 12 -443 val_443 2008-12-31 12 -323 val_323 2008-12-31 12 -325 val_325 2008-12-31 12 -277 val_277 2008-12-31 12 -230 val_230 2008-12-31 12 -478 val_478 2008-12-31 12 -178 val_178 2008-12-31 12 -468 val_468 2008-12-31 12 -310 val_310 2008-12-31 12 -317 val_317 2008-12-31 12 -333 val_333 2008-12-31 12 -493 val_493 2008-12-31 12 -460 val_460 2008-12-31 12 -207 val_207 2008-12-31 12 -249 val_249 2008-12-31 12 -265 val_265 2008-12-31 12 -480 val_480 2008-12-31 12 -83 val_83 2008-12-31 12 -136 val_136 2008-12-31 12 -353 val_353 2008-12-31 12 -172 val_172 2008-12-31 12 -214 val_214 2008-12-31 12 -462 val_462 2008-12-31 12 -233 val_233 2008-12-31 12 -406 val_406 2008-12-31 12 -133 val_133 2008-12-31 12 -175 val_175 2008-12-31 12 -189 val_189 2008-12-31 12 -454 val_454 2008-12-31 12 -375 val_375 2008-12-31 12 -401 val_401 2008-12-31 12 -421 val_421 2008-12-31 12 -407 val_407 2008-12-31 12 -384 val_384 2008-12-31 12 -256 val_256 2008-12-31 12 -26 val_26 2008-12-31 12 -134 val_134 2008-12-31 12 -67 val_67 2008-12-31 12 -384 val_384 2008-12-31 12 -379 val_379 2008-12-31 12 -18 val_18 2008-12-31 12 -462 val_462 2008-12-31 12 -492 val_492 2008-12-31 12 -100 val_100 2008-12-31 12 -298 val_298 2008-12-31 12 -9 val_9 2008-12-31 12 -341 val_341 2008-12-31 12 -498 val_498 2008-12-31 12 -146 val_146 2008-12-31 12 -458 val_458 2008-12-31 12 -362 val_362 2008-12-31 12 -186 val_186 2008-12-31 12 -285 val_285 2008-12-31 12 -348 val_348 2008-12-31 12 -167 val_167 2008-12-31 12 -18 val_18 2008-12-31 12 -273 val_273 2008-12-31 12 -183 val_183 2008-12-31 12 -281 val_281 2008-12-31 12 -344 val_344 2008-12-31 12 -97 val_97 2008-12-31 12 -469 val_469 2008-12-31 12 -315 val_315 2008-12-31 12 -84 val_84 2008-12-31 12 -28 val_28 2008-12-31 12 -37 val_37 2008-12-31 12 -448 val_448 2008-12-31 12 -152 val_152 2008-12-31 12 -348 val_348 2008-12-31 12 -307 val_307 2008-12-31 12 -194 val_194 2008-12-31 12 -414 val_414 2008-12-31 12 -477 val_477 2008-12-31 12 -222 val_222 2008-12-31 12 -126 val_126 2008-12-31 12 -90 val_90 2008-12-31 12 -169 val_169 2008-12-31 12 -403 val_403 2008-12-31 12 -400 val_400 2008-12-31 12 -200 val_200 2008-12-31 12 -97 val_97 2008-12-31 12 diff --git a/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c b/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c deleted file mode 100644 index 2ac44ee23f45c..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: nzhang_part1 -CreateTime: Tue Jan 28 02:18:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 b/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 deleted file mode 100644 index f0b07ca89fcb3..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: nzhang_part1 -CreateTime: Tue Jan 28 02:18:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 b/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 deleted file mode 100644 index f94614124bcf6..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-12-31, 11] -Database: default -Table: nzhang_part2 -CreateTime: Tue Jan 28 02:18:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2/ds=2008-12-31/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 b/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 deleted file mode 100644 index f00fb5b3bd491..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-12-31, 12] -Database: default -Table: nzhang_part2 -CreateTime: Tue Jan 28 02:18:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2/ds=2008-12-31/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 b/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 deleted file mode 100644 index 7335c8d32fa68..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:18:17 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1 -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 1000 - p3 v3 - rawDataSize 10624 - totalSize 11624 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 b/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 deleted file mode 100644 index 674cd9b0b64e5..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:18:17 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2 -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 1000 - p3 v3 - rawDataSize 10624 - totalSize 11624 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd b/sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c b/sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd b/sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c b/sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d b/sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 b/sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f b/sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 b/sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df b/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df deleted file mode 100644 index c9a75eb6639d4..0000000000000 --- a/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_src))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_src - TableScan - alias: analyze_src - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 b/sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 b/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 deleted file mode 100644 index e622a0d3704a1..0000000000000 --- a/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 +++ /dev/null @@ -1,33 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:18:36 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_src -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 0 - numRows 500 - p3 v3 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904324 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 deleted file mode 100644 index bf589ab894312..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:18:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1390904332 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c deleted file mode 100644 index e6170a33d7de7..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:18:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12 -Partition Parameters: - transient_lastDdlTime 1390904332 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index 45f6b344b04ca..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:18:45 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 1000 - p3 v3 - rawDataSize 10624 - totalSize 11624 - transient_lastDdlTime 1390904348 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce b/sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 b/sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 deleted file mode 100644 index 9ff40db416cb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:18:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904340 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index 5c13e8793cc76..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:18:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904348 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index bafa551db56e9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,38 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 14 14:57:42 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 1000 - rawDataSize 10624 - totalSize 11624 - transient_lastDdlTime 1389740280 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 b/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 deleted file mode 100644 index d5341825cff70..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 +++ /dev/null @@ -1,18 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 b/sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 deleted file mode 100644 index 7085e9397e59b..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:57:50 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1389740280 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index a3eb8c6466032..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:57:50 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1389740280 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 b/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 deleted file mode 100644 index d1e0c6243244d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 12)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 b/sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index 27ff9a687f5c6..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904374 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d b/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d deleted file mode 100644 index ce340cced8844..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09') (TOK_PARTVAL hr 11)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 b/sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 deleted file mode 100644 index 649828a63174b..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904382 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 b/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 deleted file mode 100644 index e4c8bf3e0005f..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09') (TOK_PARTVAL hr 12)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 b/sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c deleted file mode 100644 index 74aeceec37398..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904390 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 b/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 deleted file mode 100644 index 25d9c86f74634..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds) (TOK_PARTVAL hr)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 b/sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 deleted file mode 100644 index 5939c257f2f67..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index 8dc22de8e0ee6..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 deleted file mode 100644 index a19f38cc6a364..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c deleted file mode 100644 index aead24acd518c..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index 5f8bf1774f144..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:19:10 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 4 - numPartitions 4 - numRows 2000 - p3 v3 - rawDataSize 21248 - totalSize 23248 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 b/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 deleted file mode 100644 index d30acbf86a295..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce b/sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 deleted file mode 100644 index 07a61fc1a8bd1..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904366 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index bd16e76d28d83..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:19:10 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 1 - numRows 500 - p3 v3 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904366 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6f3df708fa339df236ec9375d2ad37fa similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c rename to sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6f3df708fa339df236ec9375d2ad37fa diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-36eee5cbac5c0c3228e499805b32f6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-36eee5cbac5c0c3228e499805b32f6 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 +++ b/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 b/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 +++ b/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 b/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 index 4475cb837eaa4..e01e7187693ed 100644 --- a/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 +++ b/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 @@ -1,27 +1,28 @@ # col_name data_type comment -key string None -value string None +key string +value string # Partition Information # col_name data_type comment -part string None +part string # Detailed Partition Information Partition Value: [1] Database: default Table: tmptable -CreateTime: Sun Jan 05 00:32:00 PST 2014 +CreateTime: Tue Oct 21 05:20:33 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2020775722466758355/tmptable/part=1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmptable/part=1 Partition Parameters: + COLUMN_STATS_ACCURATE true numFiles 1 numRows 0 rawDataSize 0 totalSize 0 - transient_lastDdlTime 1388910720 + transient_lastDdlTime 1413894033 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -32,4 +33,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index 5949a76d84a11..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:58:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 0 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1389740295 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 deleted file mode 100644 index 97f5929fd816a..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:58:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1389740294 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c deleted file mode 100644 index 6d08ff47abc2c..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:58:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-09/hr=12 -Partition Parameters: - transient_lastDdlTime 1389740294 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index f441c8b483868..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,38 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 14 14:58:04 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 0 - rawDataSize 0 - totalSize 11624 - transient_lastDdlTime 1389740295 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab b/sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 b/sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 b/sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 b/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 deleted file mode 100644 index f263eed117b10..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08'))) noscan) - -STAGE DEPENDENCIES: - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-1 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 b/sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e b/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e deleted file mode 100644 index 6a115f7e6e61a..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial -CreateTime: Tue Jan 14 14:58:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 0 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1389740307 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f b/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f deleted file mode 100644 index 1d96413b915da..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart_partial -CreateTime: Tue Jan 14 14:58:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 0 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1389740307 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 b/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 deleted file mode 100644 index 4c4380e63b7f9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart_partial -CreateTime: Tue Jan 14 14:58:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1389740307 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a b/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a deleted file mode 100644 index ba90d8d3d2477..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart_partial -CreateTime: Tue Jan 14 14:58:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-09/hr=12 -Partition Parameters: - transient_lastDdlTime 1389740307 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 b/sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c b/sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 b/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 deleted file mode 100644 index b671e68cc4c3b..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) noscan) - -STAGE DEPENDENCIES: - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-1 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e b/sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae b/sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 b/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 deleted file mode 100644 index 96499c0fb0f7e..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:58:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 0 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1389740294 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 b/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 deleted file mode 100644 index b5fc469438c83..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 14 14:36:56 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11 -Partition Parameters: - transient_lastDdlTime 1389739016 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef b/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef deleted file mode 100644 index cc58ef9026786..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef +++ /dev/null @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial_scan) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) partialscan) - -STAGE DEPENDENCIES: - Stage-2 is a root stage - Stage-1 depends on stages: Stage-2 - -STAGE PLANS: - Stage: Stage-2 - Partial Scan Statistics - - Stage: Stage-1 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 b/sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c b/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c deleted file mode 100644 index 3243fe9fb497d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 14 14:36:56 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 4812 - totalSize 7456 - transient_lastDdlTime 1389739019 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 b/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 deleted file mode 100644 index 6accd64d06d69..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 14 14:36:56 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1389739016 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 b/sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 b/sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 b/sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 deleted file mode 100644 index 942b7cfe07235..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 28 02:20:21 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11 -Partition Parameters: - transient_lastDdlTime 1390904421 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef b/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef deleted file mode 100644 index 9c704a6ef4126..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef +++ /dev/null @@ -1,15 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial_scan) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) partialscan) - -STAGE DEPENDENCIES: - Stage-2 is a root stage - Stage-1 depends on stages: Stage-2 - -STAGE PLANS: - Stage: Stage-2 - Partial Scan Statistics - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c b/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c deleted file mode 100644 index feee75f095d0d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 28 02:20:21 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 4812 - totalSize 7456 - transient_lastDdlTime 1390904425 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 deleted file mode 100644 index f39d366764c95..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 28 02:20:21 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1390904421 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca b/sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-36eee5cbac5c0c3228e499805b32f6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/stats_publisher_error_1-1-36eee5cbac5c0c3228e499805b32f6 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 b/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 deleted file mode 100644 index 9bed96d855476..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 +++ /dev/null @@ -1 +0,0 @@ -str_to_map(text, delimiter1, delimiter2) - Creates a map by parsing text diff --git a/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c b/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c deleted file mode 100644 index 1da11fbfa31d5..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c +++ /dev/null @@ -1,2 +0,0 @@ -str_to_map(text, delimiter1, delimiter2) - Creates a map by parsing text -Split text into key-value pairs using two delimiters. The first delimiter seperates pairs, and the second delimiter sperates key and value. If only one parameter is given, default delimiters are used: ',' as delimiter1 and '=' as delimiter2. diff --git a/sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 b/sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a b/sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e b/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e deleted file mode 100644 index cd4d4a384163b..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e +++ /dev/null @@ -1,3 +0,0 @@ -444 -444 -444 diff --git a/sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 b/sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec b/sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 b/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 deleted file mode 100644 index e8183f05f5db6..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 +++ /dev/null @@ -1,3 +0,0 @@ -1 -1 -1 diff --git a/sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 b/sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda b/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda deleted file mode 100644 index 62813f9d10491..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda +++ /dev/null @@ -1,3 +0,0 @@ -{"b":"2","a":"1","c":"3"} -{"b":"2","a":"1","c":"3"} -{"b":"2","a":"1","c":"3"} diff --git a/sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf b/sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad b/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad deleted file mode 100644 index 62813f9d10491..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad +++ /dev/null @@ -1,3 +0,0 @@ -{"b":"2","a":"1","c":"3"} -{"b":"2","a":"1","c":"3"} -{"b":"2","a":"1","c":"3"} diff --git a/sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 b/sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd b/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd deleted file mode 100644 index e8183f05f5db6..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd +++ /dev/null @@ -1,3 +0,0 @@ -1 -1 -1 diff --git a/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 b/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 index 8938b37682023..cfb0572d8663d 100644 --- a/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 +++ b/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 @@ -497,4 +497,4 @@ test test test test -test \ No newline at end of file +test diff --git a/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca b/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca index 5b3f76a5f7797..0bc48337e2dc0 100644 --- a/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca +++ b/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca @@ -255,4 +255,4 @@ 495 1 496 1 497 1 -498 3 \ No newline at end of file +498 3 diff --git a/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 b/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 +++ b/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 b/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a b/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 b/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 b/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b b/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 b/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 b/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 b/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e b/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e +++ b/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 b/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 index 92cca3b6f1e24..95e2ae6a0fd50 100644 --- a/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 +++ b/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 @@ -1,2 +1,2 @@ -a int None -b string None \ No newline at end of file +a int +b string diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 b/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 +++ b/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 b/sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c b/sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 b/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/timestamp_1-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/timestamp_1-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 b/sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 b/sql/hive/src/test/resources/golden/timestamp_1-1-d362501d0176855077e65f8faf067fa8 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 rename to sql/hive/src/test/resources/golden/timestamp_1-1-d362501d0176855077e65f8faf067fa8 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-10-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-10-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-11-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-11-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 b/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-12-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-12-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d b/sql/hive/src/test/resources/golden/timestamp_1-13-d242038c04dd4ee6075c7eebc0f75f17 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d rename to sql/hive/src/test/resources/golden/timestamp_1-13-d242038c04dd4ee6075c7eebc0f75f17 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-14-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/timestamp_1-14-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-15-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-15-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-16-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-16-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-17-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-17-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-19-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-19-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d b/sql/hive/src/test/resources/golden/timestamp_1-2-1d7cf3a2512fa1876b422b79bbe05426 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d rename to sql/hive/src/test/resources/golden/timestamp_1-2-1d7cf3a2512fa1876b422b79bbe05426 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 b/sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-20-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-20-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-21-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-21-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e b/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 b/sql/hive/src/test/resources/golden/timestamp_1-22-cdb04b49b836e0244f6f0857aea7da8a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 rename to sql/hive/src/test/resources/golden/timestamp_1-22-cdb04b49b836e0244f6f0857aea7da8a diff --git a/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-23-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/timestamp_1-23-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-24-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-24-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-25-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-25-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-26-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-26-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-28-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-28-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-29-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-29-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c b/sql/hive/src/test/resources/golden/timestamp_1-3-74f477a8b726f5193dd42ef378a793c4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c rename to sql/hive/src/test/resources/golden/timestamp_1-3-74f477a8b726f5193dd42ef378a793c4 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 b/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 b/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-30-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-30-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/timestamp_1-31-9587976bd7e6caa5b667975c14e8dd53 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 rename to sql/hive/src/test/resources/golden/timestamp_1-31-9587976bd7e6caa5b667975c14e8dd53 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-32-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/timestamp_1-32-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-33-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-33-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-34-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-34-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-35-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-35-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-37-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-37-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-38-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-38-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d b/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-39-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-39-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea b/sql/hive/src/test/resources/golden/timestamp_1-4-d833b177fac3162215468dde991f71d1 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea rename to sql/hive/src/test/resources/golden/timestamp_1-4-d833b177fac3162215468dde991f71d1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 b/sql/hive/src/test/resources/golden/timestamp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 rename to sql/hive/src/test/resources/golden/timestamp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b diff --git a/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-41-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_1-41-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-42-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-42-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-43-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-43-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-44-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-44-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-46-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-46-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-47-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-47-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 b/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-48-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-48-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 b/sql/hive/src/test/resources/golden/timestamp_1-49-7a59f9f939efc4b96f8159d00b39ed3 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 rename to sql/hive/src/test/resources/golden/timestamp_1-49-7a59f9f939efc4b96f8159d00b39ed3 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-5-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_1-5-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-50-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_1-50-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-51-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-51-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-52-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-52-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-53-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-53-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-55-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-55-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-56-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-56-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-57-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-57-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 b/sql/hive/src/test/resources/golden/timestamp_1-58-d362501d0176855077e65f8faf067fa8 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 rename to sql/hive/src/test/resources/golden/timestamp_1-58-d362501d0176855077e65f8faf067fa8 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-8-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-8-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/timestamp_2-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/timestamp_2-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 b/sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 b/sql/hive/src/test/resources/golden/timestamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 rename to sql/hive/src/test/resources/golden/timestamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-10-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-10-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-11-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-11-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-12-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-12-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd b/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 b/sql/hive/src/test/resources/golden/timestamp_2-13-5f450162886ccc79af149541527f5643 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 rename to sql/hive/src/test/resources/golden/timestamp_2-13-5f450162886ccc79af149541527f5643 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-14-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_2-14-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-15-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-15-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-16-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-16-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-17-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-17-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-19-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-19-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 b/sql/hive/src/test/resources/golden/timestamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 rename to sql/hive/src/test/resources/golden/timestamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 b/sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-20-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-20-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-21-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-21-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b b/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 b/sql/hive/src/test/resources/golden/timestamp_2-22-469fe94fb60f4b00809190c303434641 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 rename to sql/hive/src/test/resources/golden/timestamp_2-22-469fe94fb60f4b00809190c303434641 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-23-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_2-23-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-24-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-24-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-25-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-25-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-26-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-26-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-28-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-28-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-29-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-29-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 b/sql/hive/src/test/resources/golden/timestamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 rename to sql/hive/src/test/resources/golden/timestamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 b/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-30-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-30-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 b/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f b/sql/hive/src/test/resources/golden/timestamp_2-31-8f506498acf0c99c30960a00981ef460 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f rename to sql/hive/src/test/resources/golden/timestamp_2-31-8f506498acf0c99c30960a00981ef460 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-32-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_2-32-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-33-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-33-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-34-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-34-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-35-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-35-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-37-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-37-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-38-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-38-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-39-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-39-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 b/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 b/sql/hive/src/test/resources/golden/timestamp_2-4-81d6d29dcb3fd12a519426dff64411d2 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 rename to sql/hive/src/test/resources/golden/timestamp_2-4-81d6d29dcb3fd12a519426dff64411d2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 b/sql/hive/src/test/resources/golden/timestamp_2-40-972a007e54d1c09e9ac9549c19a32dbb similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 rename to sql/hive/src/test/resources/golden/timestamp_2-40-972a007e54d1c09e9ac9549c19a32dbb diff --git a/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 b/sql/hive/src/test/resources/golden/timestamp_2-41-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 rename to sql/hive/src/test/resources/golden/timestamp_2-41-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-42-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-42-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-43-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-43-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-44-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-44-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-46-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-46-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-47-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-47-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-48-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-48-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb b/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f b/sql/hive/src/test/resources/golden/timestamp_2-49-650d2727b007638e0ed39b37c9498d66 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f rename to sql/hive/src/test/resources/golden/timestamp_2-49-650d2727b007638e0ed39b37c9498d66 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 b/sql/hive/src/test/resources/golden/timestamp_2-5-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 rename to sql/hive/src/test/resources/golden/timestamp_2-5-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 b/sql/hive/src/test/resources/golden/timestamp_2-50-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 rename to sql/hive/src/test/resources/golden/timestamp_2-50-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-51-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-51-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-52-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-52-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-53-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-53-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-55-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-55-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-56-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-56-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-57-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-57-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef b/sql/hive/src/test/resources/golden/timestamp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef rename to sql/hive/src/test/resources/golden/timestamp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-6-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-6-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-7-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-7-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/timestamp_3-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/timestamp_3-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 b/sql/hive/src/test/resources/golden/timestamp_3-1-165256158e3db1ce19c3c9db3c8011d2 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 rename to sql/hive/src/test/resources/golden/timestamp_3-1-165256158e3db1ce19c3c9db3c8011d2 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b b/sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b b/sql/hive/src/test/resources/golden/timestamp_3-10-ffc79abb874323e165963aa39f460a9b similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b rename to sql/hive/src/test/resources/golden/timestamp_3-10-ffc79abb874323e165963aa39f460a9b diff --git a/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 b/sql/hive/src/test/resources/golden/timestamp_3-11-7b1ec929239ee305ea9da46ebb990c67 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 rename to sql/hive/src/test/resources/golden/timestamp_3-11-7b1ec929239ee305ea9da46ebb990c67 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a b/sql/hive/src/test/resources/golden/timestamp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a rename to sql/hive/src/test/resources/golden/timestamp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a diff --git a/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 b/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 new file mode 100644 index 0000000000000..4cfaa1b324da3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 @@ -0,0 +1 @@ +2011-04-29 20:46:56.4485 1.3041352164485E9 1 1.3041352164485E9 1.3041352164485E9 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 b/sql/hive/src/test/resources/golden/timestamp_3-14-165256158e3db1ce19c3c9db3c8011d2 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 rename to sql/hive/src/test/resources/golden/timestamp_3-14-165256158e3db1ce19c3c9db3c8011d2 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 b/sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d b/sql/hive/src/test/resources/golden/timestamp_3-2-81edf5107270547641586aa02b4e7d9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d rename to sql/hive/src/test/resources/golden/timestamp_3-2-81edf5107270547641586aa02b4e7d9b diff --git a/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a b/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b b/sql/hive/src/test/resources/golden/timestamp_3-3-7a012a0d98729da25b5ac374855dcee4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b rename to sql/hive/src/test/resources/golden/timestamp_3-3-7a012a0d98729da25b5ac374855dcee4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/timestamp_3-4-86514381187b246a5685577c1968c559 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad rename to sql/hive/src/test/resources/golden/timestamp_3-4-86514381187b246a5685577c1968c559 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 b/sql/hive/src/test/resources/golden/timestamp_3-5-935d0d2492beab99bbbba26ba62a1db4 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 rename to sql/hive/src/test/resources/golden/timestamp_3-5-935d0d2492beab99bbbba26ba62a1db4 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 b/sql/hive/src/test/resources/golden/timestamp_3-6-8fe348d5d9b9903a26eda32d308b8e41 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 rename to sql/hive/src/test/resources/golden/timestamp_3-6-8fe348d5d9b9903a26eda32d308b8e41 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 b/sql/hive/src/test/resources/golden/timestamp_3-7-6be5fe01c502cd24db32a3781c97a703 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 rename to sql/hive/src/test/resources/golden/timestamp_3-7-6be5fe01c502cd24db32a3781c97a703 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add b/sql/hive/src/test/resources/golden/timestamp_3-8-6066ba0451cd0fcfac4bea6376e72add similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add rename to sql/hive/src/test/resources/golden/timestamp_3-8-6066ba0451cd0fcfac4bea6376e72add diff --git a/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e b/sql/hive/src/test/resources/golden/timestamp_3-9-22e03daa775eab145d39ec0730953f7e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e rename to sql/hive/src/test/resources/golden/timestamp_3-9-22e03daa775eab145d39ec0730953f7e diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/timestamp_comparison-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/timestamp_comparison-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 b/sql/hive/src/test/resources/golden/timestamp_comparison-1-4b68f7ad0f8cf337e42bf16a45e15818 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 rename to sql/hive/src/test/resources/golden/timestamp_comparison-1-4b68f7ad0f8cf337e42bf16a45e15818 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d b/sql/hive/src/test/resources/golden/timestamp_comparison-2-60557e7bd2822c89fa8b076a9d0520fc similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d rename to sql/hive/src/test/resources/golden/timestamp_comparison-2-60557e7bd2822c89fa8b076a9d0520fc diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc b/sql/hive/src/test/resources/golden/timestamp_comparison-3-f96a9d88327951bd93f672dc2463ecd4 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc rename to sql/hive/src/test/resources/golden/timestamp_comparison-3-f96a9d88327951bd93f672dc2463ecd4 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 b/sql/hive/src/test/resources/golden/timestamp_comparison-4-13e17ed811165196416f777cbc162592 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 rename to sql/hive/src/test/resources/golden/timestamp_comparison-4-13e17ed811165196416f777cbc162592 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 b/sql/hive/src/test/resources/golden/timestamp_comparison-5-4fa8a36edbefde4427c2ab2cf30e6399 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 rename to sql/hive/src/test/resources/golden/timestamp_comparison-5-4fa8a36edbefde4427c2ab2cf30e6399 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 b/sql/hive/src/test/resources/golden/timestamp_comparison-6-7e4fb6e8ba01df422e4c67e06a0c8453 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 rename to sql/hive/src/test/resources/golden/timestamp_comparison-6-7e4fb6e8ba01df422e4c67e06a0c8453 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 b/sql/hive/src/test/resources/golden/timestamp_comparison-7-8c8e73673a950f6b3d960b08fcea076f similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 rename to sql/hive/src/test/resources/golden/timestamp_comparison-7-8c8e73673a950f6b3d960b08fcea076f diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 b/sql/hive/src/test/resources/golden/timestamp_comparison-8-510c0a2a57dc5df8588bd13c4152f8bc similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 rename to sql/hive/src/test/resources/golden/timestamp_comparison-8-510c0a2a57dc5df8588bd13c4152f8bc diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 b/sql/hive/src/test/resources/golden/timestamp_comparison-9-659d5b1ae8200f13f265270e52a3dd65 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 rename to sql/hive/src/test/resources/golden/timestamp_comparison-9-659d5b1ae8200f13f265270e52a3dd65 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 b/sql/hive/src/test/resources/golden/timestamp_lazy-2-bb5a4a13274290029bd07d95c2f92563 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 rename to sql/hive/src/test/resources/golden/timestamp_lazy-2-bb5a4a13274290029bd07d95c2f92563 diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 b/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c b/sql/hive/src/test/resources/golden/timestamp_null-2-51762cf5079877abf7d81127738f4e5 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c rename to sql/hive/src/test/resources/golden/timestamp_null-2-51762cf5079877abf7d81127738f4e5 diff --git a/sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 b/sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/timestamp_udf-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/timestamp_udf-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af b/sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d b/sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a b/sql/hive/src/test/resources/golden/timestamp_udf-1-79914c5347620c6e62a8e0b9a95984af similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a rename to sql/hive/src/test/resources/golden/timestamp_udf-1-79914c5347620c6e62a8e0b9a95984af diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-18-cb033ecad964a2623bc633ac1d3f752a b/sql/hive/src/test/resources/golden/timestamp_udf-10-287614364eaa3fb82aad08c6b62cc938 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-18-cb033ecad964a2623bc633ac1d3f752a rename to sql/hive/src/test/resources/golden/timestamp_udf-10-287614364eaa3fb82aad08c6b62cc938 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-10-dbc23736a61d9482d13cacada02a7a09 b/sql/hive/src/test/resources/golden/timestamp_udf-11-dbc23736a61d9482d13cacada02a7a09 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-10-dbc23736a61d9482d13cacada02a7a09 rename to sql/hive/src/test/resources/golden/timestamp_udf-11-dbc23736a61d9482d13cacada02a7a09 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-11-442cf850a0cc1f1dcfdeaeffbffb2c35 b/sql/hive/src/test/resources/golden/timestamp_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-11-442cf850a0cc1f1dcfdeaeffbffb2c35 rename to sql/hive/src/test/resources/golden/timestamp_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-12-51959036fd4ac4f1e24f4e06eb9b0b6 b/sql/hive/src/test/resources/golden/timestamp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-12-51959036fd4ac4f1e24f4e06eb9b0b6 rename to sql/hive/src/test/resources/golden/timestamp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-13-6ab3f356deaf807e8accc37e1f4849a b/sql/hive/src/test/resources/golden/timestamp_udf-14-6ab3f356deaf807e8accc37e1f4849a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-13-6ab3f356deaf807e8accc37e1f4849a rename to sql/hive/src/test/resources/golden/timestamp_udf-14-6ab3f356deaf807e8accc37e1f4849a diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-14-c745a1016461403526d44928a269c1de b/sql/hive/src/test/resources/golden/timestamp_udf-15-c745a1016461403526d44928a269c1de similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-14-c745a1016461403526d44928a269c1de rename to sql/hive/src/test/resources/golden/timestamp_udf-15-c745a1016461403526d44928a269c1de diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-15-7ab76c4458c7f78038c8b1df0fdeafbe b/sql/hive/src/test/resources/golden/timestamp_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-15-7ab76c4458c7f78038c8b1df0fdeafbe rename to sql/hive/src/test/resources/golden/timestamp_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-16-b36e87e17ca24d82072220bff559c718 b/sql/hive/src/test/resources/golden/timestamp_udf-17-b36e87e17ca24d82072220bff559c718 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-16-b36e87e17ca24d82072220bff559c718 rename to sql/hive/src/test/resources/golden/timestamp_udf-17-b36e87e17ca24d82072220bff559c718 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-17-dad44d2d4a421286e9da080271bd2639 b/sql/hive/src/test/resources/golden/timestamp_udf-18-dad44d2d4a421286e9da080271bd2639 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-17-dad44d2d4a421286e9da080271bd2639 rename to sql/hive/src/test/resources/golden/timestamp_udf-18-dad44d2d4a421286e9da080271bd2639 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-19-79914c5347620c6e62a8e0b9a95984af b/sql/hive/src/test/resources/golden/timestamp_udf-19-79914c5347620c6e62a8e0b9a95984af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-9-287614364eaa3fb82aad08c6b62cc938 b/sql/hive/src/test/resources/golden/timestamp_udf-19-cb033ecad964a2623bc633ac1d3f752a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-9-287614364eaa3fb82aad08c6b62cc938 rename to sql/hive/src/test/resources/golden/timestamp_udf-19-cb033ecad964a2623bc633ac1d3f752a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 b/sql/hive/src/test/resources/golden/timestamp_udf-2-59fc1842a23369235d42ed040d45fb3d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 rename to sql/hive/src/test/resources/golden/timestamp_udf-2-59fc1842a23369235d42ed040d45fb3d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 b/sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-20-59fc1842a23369235d42ed040d45fb3d b/sql/hive/src/test/resources/golden/timestamp_udf-20-59fc1842a23369235d42ed040d45fb3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 b/sql/hive/src/test/resources/golden/timestamp_udf-20-79914c5347620c6e62a8e0b9a95984af similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 rename to sql/hive/src/test/resources/golden/timestamp_udf-20-79914c5347620c6e62a8e0b9a95984af diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa b/sql/hive/src/test/resources/golden/timestamp_udf-21-59fc1842a23369235d42ed040d45fb3d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa rename to sql/hive/src/test/resources/golden/timestamp_udf-21-59fc1842a23369235d42ed040d45fb3d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 b/sql/hive/src/test/resources/golden/timestamp_udf-3-9039f474f9a96e9f15ace528faeed923 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 rename to sql/hive/src/test/resources/golden/timestamp_udf-3-9039f474f9a96e9f15ace528faeed923 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f b/sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-4-80ce02ec84ee8abcb046367ca37279cc b/sql/hive/src/test/resources/golden/timestamp_udf-4-80ce02ec84ee8abcb046367ca37279cc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/timestamp_udf-4-b0fd4ca3b22eb732a32772399331352f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/timestamp_udf-4-b0fd4ca3b22eb732a32772399331352f diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 b/sql/hive/src/test/resources/golden/timestamp_udf-5-66868a2b075de978784011e9955483d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 rename to sql/hive/src/test/resources/golden/timestamp_udf-5-66868a2b075de978784011e9955483d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-5-1124399033bcadf3874fb48f593392d b/sql/hive/src/test/resources/golden/timestamp_udf-6-1124399033bcadf3874fb48f593392d similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-5-1124399033bcadf3874fb48f593392d rename to sql/hive/src/test/resources/golden/timestamp_udf-6-1124399033bcadf3874fb48f593392d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-6-5810193ce35d38c23f4fc4b4979d60a4 b/sql/hive/src/test/resources/golden/timestamp_udf-7-5810193ce35d38c23f4fc4b4979d60a4 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-6-5810193ce35d38c23f4fc4b4979d60a4 rename to sql/hive/src/test/resources/golden/timestamp_udf-7-5810193ce35d38c23f4fc4b4979d60a4 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-7-250e640a6a818f989f3f3280b00f64f9 b/sql/hive/src/test/resources/golden/timestamp_udf-8-250e640a6a818f989f3f3280b00f64f9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-7-250e640a6a818f989f3f3280b00f64f9 rename to sql/hive/src/test/resources/golden/timestamp_udf-8-250e640a6a818f989f3f3280b00f64f9 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-8-975df43df015d86422965af456f87a94 b/sql/hive/src/test/resources/golden/timestamp_udf-9-975df43df015d86422965af456f87a94 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-8-975df43df015d86422965af456f87a94 rename to sql/hive/src/test/resources/golden/timestamp_udf-9-975df43df015d86422965af456f87a94 diff --git a/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe b/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe index d23e05acf7ba5..e34118512c1d7 100644 --- a/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe +++ b/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe @@ -497,4 +497,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 b/sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 b/sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 b/sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 b/sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 b/sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 b/sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f b/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f deleted file mode 100644 index c6f628b1a3eef..0000000000000 --- a/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f +++ /dev/null @@ -1 +0,0 @@ -[0,1,2] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 b/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 index 839efe2d57956..9370a501a5179 100644 --- a/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 +++ b/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 @@ -165,4 +165,4 @@ 98 val_98 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 b/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 index 839efe2d57956..9370a501a5179 100644 --- a/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 +++ b/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 @@ -165,4 +165,4 @@ 98 val_98 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d b/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d index 66fafbdf72b63..3df4716f0b05f 100644 --- a/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d +++ b/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 200 val_200 200 val_200 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 b/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 index 66fafbdf72b63..3df4716f0b05f 100644 --- a/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 +++ b/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 200 val_200 200 val_200 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/type_cast_1-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/type_cast_1-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d b/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 b/sql/hive/src/test/resources/golden/type_cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 rename to sql/hive/src/test/resources/golden/type_cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d diff --git a/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 b/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/type_widening-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/type_widening-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 b/sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 b/sql/hive/src/test/resources/golden/type_widening-1-630ac2c7e7dea4837384ccd572209229 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 rename to sql/hive/src/test/resources/golden/type_widening-1-630ac2c7e7dea4837384ccd572209229 diff --git a/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 b/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 b/sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/type_widening-2-cfbdf2b6fca84c6e23d4e691d2221bd6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/type_widening-2-cfbdf2b6fca84c6e23d4e691d2221bd6 diff --git a/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 b/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 deleted file mode 100644 index cf940f4c5faa8..0000000000000 --- a/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 +++ /dev/null @@ -1,1000 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c b/sql/hive/src/test/resources/golden/type_widening-3-a0ba6952d9bf830d1d1ea7aebd3784a2 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c rename to sql/hive/src/test/resources/golden/type_widening-3-a0ba6952d9bf830d1d1ea7aebd3784a2 diff --git a/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 b/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 new file mode 100644 index 0000000000000..00841d23b3f94 --- /dev/null +++ b/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 @@ -0,0 +1,1000 @@ +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e b/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e index fed3d6802023a..bb3393324db7a 100644 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e @@ -1 +1 @@ -collect_set(x) - Returns a set of objects with duplicate elements eliminated \ No newline at end of file +collect_set(x) - Returns a set of objects with duplicate elements eliminated diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 b/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 index fed3d6802023a..bb3393324db7a 100644 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 @@ -1 +1 @@ -collect_set(x) - Returns a set of objects with duplicate elements eliminated \ No newline at end of file +collect_set(x) - Returns a set of objects with duplicate elements eliminated diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db index cedc3068ee4a1..c87ba74c9000a 100644 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db @@ -17,4 +17,4 @@ 27 ["val_27"] 28 ["val_28"] 30 ["val_30"] -33 ["val_33"] \ No newline at end of file +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 b/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 new file mode 100644 index 0000000000000..337e96635cc70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 @@ -0,0 +1,20 @@ +0 ["val_0","val_0","val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5","val_5","val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12","val_12"] +15 ["val_15","val_15"] +17 ["val_17"] +18 ["val_18","val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24","val_24"] +26 ["val_26","val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-12-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udaf_collect_set-12-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-13-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udaf_collect_set-13-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db deleted file mode 100644 index cedc3068ee4a1..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db +++ /dev/null @@ -1,20 +0,0 @@ -0 ["val_0"] -2 ["val_2"] -4 ["val_4"] -5 ["val_5"] -8 ["val_8"] -9 ["val_9"] -10 ["val_10"] -11 ["val_11"] -12 ["val_12"] -15 ["val_15"] -17 ["val_17"] -18 ["val_18"] -19 ["val_19"] -20 ["val_20"] -24 ["val_24"] -26 ["val_26"] -27 ["val_27"] -28 ["val_28"] -30 ["val_30"] -33 ["val_33"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..c87ba74c9000a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db @@ -0,0 +1,20 @@ +0 ["val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12"] +15 ["val_15"] +17 ["val_17"] +18 ["val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24"] +26 ["val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udaf_collect_set-15-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udaf_collect_set-15-863233ccd616401efb4bf83c4b9e3a52 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-16-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udaf_collect_set-16-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..c87ba74c9000a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db @@ -0,0 +1,20 @@ +0 ["val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12"] +15 ["val_15"] +17 ["val_17"] +18 ["val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24"] +26 ["val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb b/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb new file mode 100644 index 0000000000000..28abc06ee9140 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb @@ -0,0 +1 @@ +collect_list(x) - Returns a list of objects with duplicates diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 b/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 new file mode 100644 index 0000000000000..28abc06ee9140 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 @@ -0,0 +1 @@ +collect_list(x) - Returns a list of objects with duplicates diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-4-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udaf_collect_set-4-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db deleted file mode 100644 index cedc3068ee4a1..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db +++ /dev/null @@ -1,20 +0,0 @@ -0 ["val_0"] -2 ["val_2"] -4 ["val_4"] -5 ["val_5"] -8 ["val_8"] -9 ["val_9"] -10 ["val_10"] -11 ["val_11"] -12 ["val_12"] -15 ["val_15"] -17 ["val_17"] -18 ["val_18"] -19 ["val_19"] -20 ["val_20"] -24 ["val_24"] -26 ["val_26"] -27 ["val_27"] -28 ["val_28"] -30 ["val_30"] -33 ["val_33"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udaf_collect_set-5-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udaf_collect_set-5-a7dc16cb82c595b18d4258a38a304b1e diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..c87ba74c9000a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db @@ -0,0 +1,20 @@ +0 ["val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12"] +15 ["val_15"] +17 ["val_17"] +18 ["val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24"] +26 ["val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 b/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 new file mode 100644 index 0000000000000..337e96635cc70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 @@ -0,0 +1,20 @@ +0 ["val_0","val_0","val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5","val_5","val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12","val_12"] +15 ["val_15","val_15"] +17 ["val_17"] +18 ["val_18","val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24","val_24"] +26 ["val_26","val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db deleted file mode 100644 index cedc3068ee4a1..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db +++ /dev/null @@ -1,20 +0,0 @@ -0 ["val_0"] -2 ["val_2"] -4 ["val_4"] -5 ["val_5"] -8 ["val_8"] -9 ["val_9"] -10 ["val_10"] -11 ["val_11"] -12 ["val_12"] -15 ["val_15"] -17 ["val_17"] -18 ["val_18"] -19 ["val_19"] -20 ["val_20"] -24 ["val_24"] -26 ["val_26"] -27 ["val_27"] -28 ["val_28"] -30 ["val_30"] -33 ["val_33"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-8-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udaf_collect_set-8-863233ccd616401efb4bf83c4b9e3a52 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-9-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udaf_collect_set-9-a7dc16cb82c595b18d4258a38a304b1e diff --git a/sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 b/sql/hive/src/test/resources/golden/udaf_corr-2-e886f45c8f085596ffd420f89cdc2909 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 rename to sql/hive/src/test/resources/golden/udaf_corr-2-e886f45c8f085596ffd420f89cdc2909 diff --git a/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e b/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e index 6e5d422b3132e..06b8ee9e72496 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e +++ b/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e @@ -1,2 +1,2 @@ corr(x,y) - Returns the Pearson coefficient of correlation -between a set of number pairs \ No newline at end of file +between a set of number pairs diff --git a/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a b/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a index fcb49ae69f74f..f516ef3a30fdf 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a +++ b/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a @@ -6,4 +6,4 @@ a singleton set, NULL will be returned. Otherwise, it computes the following: COVAR_POP(x,y)/(STDDEV_POP(x)*STDDEV_POP(y)) where neither x nor y is null, COVAR_POP is the population covariance, -and STDDEV_POP is the population standard deviation. \ No newline at end of file +and STDDEV_POP is the population standard deviation. diff --git a/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb b/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb +++ b/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 b/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 +++ b/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 b/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 +++ b/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 b/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 index 3f730875aef8c..946c9d58047e5 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 +++ b/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 @@ -3,4 +3,4 @@ 3 NULL 4 NULL 5 NULL -6 NULL \ No newline at end of file +6 NULL diff --git a/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 b/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 index 5d97236e8b03f..011d78d68766d 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 +++ b/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 @@ -1 +1 @@ -0.6633880657639323 \ No newline at end of file +0.6633880657639323 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 b/sql/hive/src/test/resources/golden/udaf_covar_pop-2-e886f45c8f085596ffd420f89cdc2909 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 rename to sql/hive/src/test/resources/golden/udaf_covar_pop-2-e886f45c8f085596ffd420f89cdc2909 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 b/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 index 0fed030c22af3..eadc2e1aa99b7 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 @@ -1 +1 @@ -covar_pop(x,y) - Returns the population covariance of a set of number pairs \ No newline at end of file +covar_pop(x,y) - Returns the population covariance of a set of number pairs diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad b/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad index 4037062d2da93..7323e72fccc7c 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad @@ -3,4 +3,4 @@ The function takes as arguments any pair of numeric types and returns a double. Any pair with a NULL is ignored. If the function is applied to an empty set, NULL will be returned. Otherwise, it computes the following: (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/COUNT(x,y) -where neither x nor y is null. \ No newline at end of file +where neither x nor y is null. diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd b/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b b/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae b/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae index 171538eb0b00f..ba66466c2a0d0 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae @@ -1 +1 @@ -0.0 \ No newline at end of file +0.0 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f b/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f index 848e15bc61476..104018ecd43d6 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f @@ -3,4 +3,4 @@ 3 0.0 4 0.0 5 0.0 -6 0.0 \ No newline at end of file +6 0.0 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 b/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 index 1a49bf590b346..16f4e6bd601b6 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 @@ -1 +1 @@ -3.624999999999999 \ No newline at end of file +3.624999999999999 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 b/sql/hive/src/test/resources/golden/udaf_covar_samp-2-e886f45c8f085596ffd420f89cdc2909 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 rename to sql/hive/src/test/resources/golden/udaf_covar_samp-2-e886f45c8f085596ffd420f89cdc2909 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf b/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf index 97a97c8b5965a..b301d988192fd 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf @@ -1 +1 @@ -covar_samp(x,y) - Returns the sample covariance of a set of number pairs \ No newline at end of file +covar_samp(x,y) - Returns the sample covariance of a set of number pairs diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c b/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c index e666adba8df68..de059e0e79a27 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c @@ -3,4 +3,4 @@ The function takes as arguments any pair of numeric types and returns a double. Any pair with a NULL is ignored. If the function is applied to an empty set, NULL will be returned. Otherwise, it computes the following: (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/(COUNT(x,y)-1) -where neither x nor y is null. \ No newline at end of file +where neither x nor y is null. diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b b/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 b/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 b/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 index 171538eb0b00f..ba66466c2a0d0 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 @@ -1 +1 @@ -0.0 \ No newline at end of file +0.0 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a b/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a index 848e15bc61476..104018ecd43d6 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a @@ -3,4 +3,4 @@ 3 0.0 4 0.0 5 0.0 -6 0.0 \ No newline at end of file +6 0.0 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 b/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 index 30d98234a23e9..b8adc8f23da34 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 @@ -1 +1 @@ -4.833333333333332 \ No newline at end of file +4.833333333333332 diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 index 3e8bb17f24d6f..df3189a887974 100644 --- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 +++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 @@ -1 +1 @@ -[{"x":135.0284552845532,"y":246.0},{"x":381.39370078740143,"y":254.0}] \ No newline at end of file +[{"x":135.0284552845532,"y":246.0},{"x":381.39370078740143,"y":254.0}] diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 index 048770d4c8402..b1f27ead33687 100644 --- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 +++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 @@ -1 +1 @@ -[{"x":96.7349397590361,"y":166.0},{"x":257.14970059880255,"y":167.0},{"x":425.6826347305388,"y":167.0}] \ No newline at end of file +[{"x":96.7349397590361,"y":166.0},{"x":257.14970059880255,"y":167.0},{"x":425.6826347305388,"y":167.0}] diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 index 8b54db8da7167..78a46aac82c11 100644 --- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 +++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 @@ -1 +1 @@ -[{"x":9.761904761904763,"y":21.0},{"x":33.84210526315789,"y":19.0},{"x":62.75000000000001,"y":20.0},{"x":90.90322580645162,"y":31.0},{"x":122.91666666666667,"y":24.0},{"x":146.33333333333334,"y":21.0},{"x":170.70967741935485,"y":31.0},{"x":194.3571428571428,"y":28.0},{"x":214.84615384615384,"y":26.0},{"x":235.08695652173907,"y":23.0},{"x":257.80000000000007,"y":15.0},{"x":281.0333333333333,"y":30.0},{"x":298.0,"y":1.0},{"x":313.0000000000001,"y":29.0},{"x":339.5925925925926,"y":27.0},{"x":372.49999999999983,"y":24.0},{"x":402.23684210526324,"y":38.0},{"x":430.6896551724138,"y":29.0},{"x":462.32352941176464,"y":34.0},{"x":487.72413793103453,"y":29.0}] \ No newline at end of file +[{"x":9.761904761904763,"y":21.0},{"x":33.84210526315789,"y":19.0},{"x":62.75000000000001,"y":20.0},{"x":90.90322580645162,"y":31.0},{"x":122.91666666666667,"y":24.0},{"x":146.33333333333334,"y":21.0},{"x":170.70967741935485,"y":31.0},{"x":194.3571428571428,"y":28.0},{"x":214.84615384615384,"y":26.0},{"x":235.08695652173907,"y":23.0},{"x":257.80000000000007,"y":15.0},{"x":281.0333333333333,"y":30.0},{"x":298.0,"y":1.0},{"x":313.0000000000001,"y":29.0},{"x":339.5925925925926,"y":27.0},{"x":372.49999999999983,"y":24.0},{"x":402.23684210526324,"y":38.0},{"x":430.6896551724138,"y":29.0},{"x":462.32352941176464,"y":34.0},{"x":487.72413793103453,"y":29.0}] diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 index aed3f1e704750..4f7995f874388 100644 --- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 +++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 @@ -1 +1 @@ -[{"x":0.0,"y":3.0},{"x":2.0,"y":1.0},{"x":4.75,"y":4.0},{"x":8.0,"y":1.0},{"x":9.5,"y":2.0},{"x":11.666666666666666,"y":3.0},{"x":15.0,"y":2.0},{"x":17.666666666666664,"y":3.0},{"x":19.5,"y":2.0},{"x":24.0,"y":2.0},{"x":26.333333333333336,"y":3.0},{"x":28.0,"y":1.0},{"x":30.0,"y":1.0},{"x":33.0,"y":1.0},{"x":34.75,"y":4.0},{"x":37.0,"y":2.0},{"x":41.666666666666664,"y":3.0},{"x":43.5,"y":2.0},{"x":47.0,"y":1.0},{"x":51.0,"y":2.0},{"x":53.5,"y":2.0},{"x":57.666666666666664,"y":3.0},{"x":64.5,"y":2.0},{"x":66.66666666666666,"y":3.0},{"x":69.75,"y":4.0},{"x":72.0,"y":2.0},{"x":74.0,"y":1.0},{"x":76.33333333333333,"y":3.0},{"x":78.0,"y":1.0},{"x":80.0,"y":1.0},{"x":82.0,"y":1.0},{"x":83.5,"y":4.0},{"x":85.5,"y":2.0},{"x":87.0,"y":1.0},{"x":90.0,"y":3.0},{"x":92.0,"y":1.0},{"x":95.33333333333333,"y":3.0},{"x":97.5,"y":4.0},{"x":100.0,"y":2.0},{"x":103.5,"y":4.0},{"x":105.0,"y":1.0},{"x":111.0,"y":1.0},{"x":113.33333333333333,"y":3.0},{"x":116.0,"y":1.0},{"x":118.0,"y":2.0},{"x":119.4,"y":5.0},{"x":125.33333333333333,"y":3.0},{"x":128.4,"y":5.0},{"x":131.0,"y":1.0},{"x":133.66666666666666,"y":3.0},{"x":136.66666666666666,"y":3.0},{"x":138.0,"y":4.0},{"x":143.0,"y":1.0},{"x":145.66666666666666,"y":3.0},{"x":149.33333333333331,"y":3.0},{"x":152.33333333333334,"y":3.0},{"x":155.5,"y":2.0},{"x":157.5,"y":2.0},{"x":160.0,"y":1.0},{"x":162.5,"y":2.0},{"x":164.5,"y":4.0},{"x":166.75,"y":4.0},{"x":168.8,"y":5.0},{"x":170.0,"y":1.0},{"x":172.0,"y":2.0},{"x":174.5,"y":4.0},{"x":176.33333333333331,"y":3.0},{"x":178.0,"y":1.0},{"x":179.33333333333331,"y":3.0},{"x":181.0,"y":1.0},{"x":183.0,"y":1.0},{"x":186.75,"y":4.0},{"x":189.0,"y":1.0},{"x":190.66666666666666,"y":3.0},{"x":192.75,"y":4.0},{"x":194.0,"y":1.0},{"x":195.33333333333331,"y":3.0},{"x":197.0,"y":2.0},{"x":199.4,"y":5.0},{"x":201.0,"y":1.0},{"x":202.66666666666669,"y":3.0},{"x":205.0,"y":2.0},{"x":207.0,"y":2.0},{"x":208.40000000000003,"y":5.0},{"x":213.33333333333331,"y":3.0},{"x":216.0,"y":2.0},{"x":217.33333333333331,"y":3.0},{"x":219.0,"y":2.0},{"x":221.33333333333331,"y":3.0},{"x":223.5,"y":4.0},{"x":226.0,"y":1.0},{"x":228.66666666666663,"y":3.0},{"x":230.0,"y":5.0},{"x":233.0,"y":2.0},{"x":235.0,"y":1.0},{"x":237.5,"y":4.0},{"x":239.0,"y":2.0},{"x":241.66666666666669,"y":3.0},{"x":244.0,"y":1.0},{"x":247.5,"y":2.0},{"x":249.0,"y":1.0},{"x":252.0,"y":1.0},{"x":255.5,"y":4.0},{"x":257.5,"y":2.0},{"x":260.0,"y":1.0},{"x":262.5,"y":2.0},{"x":265.3333333333333,"y":3.0},{"x":272.6,"y":5.0},{"x":274.5,"y":2.0},{"x":277.3333333333333,"y":6.0},{"x":280.0,"y":2.0},{"x":281.5,"y":4.0},{"x":283.5,"y":2.0},{"x":285.0,"y":1.0},{"x":286.5,"y":2.0},{"x":288.3333333333333,"y":3.0},{"x":291.5,"y":2.0},{"x":296.0,"y":1.0},{"x":298.0,"y":3.0},{"x":302.0,"y":1.0},{"x":305.5,"y":2.0},{"x":307.3333333333333,"y":3.0},{"x":309.0,"y":2.0},{"x":310.75,"y":4.0},{"x":315.75,"y":4.0},{"x":317.6,"y":5.0},{"x":321.5,"y":4.0},{"x":323.0,"y":1.0},{"x":325.0,"y":2.0},{"x":327.0,"y":3.0},{"x":331.3333333333333,"y":3.0},{"x":333.0,"y":2.0},{"x":335.5,"y":2.0},{"x":338.5,"y":2.0},{"x":341.66666666666663,"y":3.0},{"x":344.3333333333333,"y":3.0},{"x":348.0,"y":5.0},{"x":351.0,"y":1.0},{"x":353.0,"y":2.0},{"x":356.0,"y":1.0},{"x":360.0,"y":1.0},{"x":362.0,"y":1.0},{"x":364.5,"y":2.0},{"x":366.66666666666663,"y":3.0},{"x":368.75,"y":4.0},{"x":373.5,"y":2.0},{"x":375.0,"y":1.0},{"x":377.5,"y":2.0},{"x":379.0,"y":1.0},{"x":382.0,"y":2.0},{"x":384.0,"y":3.0},{"x":386.0,"y":1.0},{"x":389.0,"y":1.0},{"x":392.0,"y":1.0},{"x":393.5,"y":2.0},{"x":395.6,"y":5.0},{"x":397.0,"y":2.0},{"x":399.0,"y":2.0},{"x":400.0,"y":1.0},{"x":401.16666666666663,"y":6.0},{"x":403.40000000000003,"y":5.0},{"x":406.20000000000005,"y":5.0},{"x":409.0,"y":3.0},{"x":411.0,"y":1.0},{"x":413.5,"y":4.0},{"x":417.0,"y":3.0},{"x":418.5,"y":2.0},{"x":421.0,"y":1.0},{"x":424.0,"y":2.0},{"x":427.0,"y":1.0},{"x":429.6,"y":5.0},{"x":431.25,"y":4.0},{"x":435.5,"y":2.0},{"x":437.75,"y":4.0},{"x":439.0,"y":2.0},{"x":443.5,"y":2.0},{"x":446.0,"y":1.0},{"x":448.5,"y":2.0},{"x":452.5,"y":2.0},{"x":454.24999999999994,"y":4.0},{"x":457.66666666666663,"y":3.0},{"x":459.33333333333337,"y":3.0},{"x":462.5,"y":4.0},{"x":466.0,"y":3.0},{"x":467.80000000000007,"y":5.0},{"x":469.16666666666663,"y":6.0},{"x":472.0,"y":1.0},{"x":475.0,"y":1.0},{"x":477.0,"y":1.0},{"x":478.33333333333326,"y":3.0},{"x":480.25,"y":4.0},{"x":482.5,"y":2.0},{"x":484.5,"y":2.0},{"x":487.0,"y":1.0},{"x":489.2,"y":5.0},{"x":491.66666666666663,"y":3.0},{"x":493.0,"y":1.0},{"x":494.5,"y":2.0},{"x":496.0,"y":1.0},{"x":497.75,"y":4.0}] \ No newline at end of file +[{"x":0.0,"y":3.0},{"x":2.0,"y":1.0},{"x":4.75,"y":4.0},{"x":8.0,"y":1.0},{"x":9.5,"y":2.0},{"x":11.666666666666666,"y":3.0},{"x":15.0,"y":2.0},{"x":17.666666666666664,"y":3.0},{"x":19.5,"y":2.0},{"x":24.0,"y":2.0},{"x":26.333333333333336,"y":3.0},{"x":28.0,"y":1.0},{"x":30.0,"y":1.0},{"x":33.0,"y":1.0},{"x":34.75,"y":4.0},{"x":37.0,"y":2.0},{"x":41.666666666666664,"y":3.0},{"x":43.5,"y":2.0},{"x":47.0,"y":1.0},{"x":51.0,"y":2.0},{"x":53.5,"y":2.0},{"x":57.666666666666664,"y":3.0},{"x":64.5,"y":2.0},{"x":66.66666666666666,"y":3.0},{"x":69.75,"y":4.0},{"x":72.0,"y":2.0},{"x":74.0,"y":1.0},{"x":76.33333333333333,"y":3.0},{"x":78.0,"y":1.0},{"x":80.0,"y":1.0},{"x":82.0,"y":1.0},{"x":83.5,"y":4.0},{"x":85.5,"y":2.0},{"x":87.0,"y":1.0},{"x":90.0,"y":3.0},{"x":92.0,"y":1.0},{"x":95.33333333333333,"y":3.0},{"x":97.5,"y":4.0},{"x":100.0,"y":2.0},{"x":103.5,"y":4.0},{"x":105.0,"y":1.0},{"x":111.0,"y":1.0},{"x":113.33333333333333,"y":3.0},{"x":116.0,"y":1.0},{"x":118.0,"y":2.0},{"x":119.4,"y":5.0},{"x":125.33333333333333,"y":3.0},{"x":128.4,"y":5.0},{"x":131.0,"y":1.0},{"x":133.66666666666666,"y":3.0},{"x":136.66666666666666,"y":3.0},{"x":138.0,"y":4.0},{"x":143.0,"y":1.0},{"x":145.66666666666666,"y":3.0},{"x":149.33333333333331,"y":3.0},{"x":152.33333333333334,"y":3.0},{"x":155.5,"y":2.0},{"x":157.5,"y":2.0},{"x":160.0,"y":1.0},{"x":162.5,"y":2.0},{"x":164.5,"y":4.0},{"x":166.75,"y":4.0},{"x":168.8,"y":5.0},{"x":170.0,"y":1.0},{"x":172.0,"y":2.0},{"x":174.5,"y":4.0},{"x":176.33333333333331,"y":3.0},{"x":178.0,"y":1.0},{"x":179.33333333333331,"y":3.0},{"x":181.0,"y":1.0},{"x":183.0,"y":1.0},{"x":186.75,"y":4.0},{"x":189.0,"y":1.0},{"x":190.66666666666666,"y":3.0},{"x":192.75,"y":4.0},{"x":194.0,"y":1.0},{"x":195.33333333333331,"y":3.0},{"x":197.0,"y":2.0},{"x":199.4,"y":5.0},{"x":201.0,"y":1.0},{"x":202.66666666666669,"y":3.0},{"x":205.0,"y":2.0},{"x":207.0,"y":2.0},{"x":208.40000000000003,"y":5.0},{"x":213.33333333333331,"y":3.0},{"x":216.0,"y":2.0},{"x":217.33333333333331,"y":3.0},{"x":219.0,"y":2.0},{"x":221.33333333333331,"y":3.0},{"x":223.5,"y":4.0},{"x":226.0,"y":1.0},{"x":228.66666666666663,"y":3.0},{"x":230.0,"y":5.0},{"x":233.0,"y":2.0},{"x":235.0,"y":1.0},{"x":237.5,"y":4.0},{"x":239.0,"y":2.0},{"x":241.66666666666669,"y":3.0},{"x":244.0,"y":1.0},{"x":247.5,"y":2.0},{"x":249.0,"y":1.0},{"x":252.0,"y":1.0},{"x":255.5,"y":4.0},{"x":257.5,"y":2.0},{"x":260.0,"y":1.0},{"x":262.5,"y":2.0},{"x":265.3333333333333,"y":3.0},{"x":272.6,"y":5.0},{"x":274.5,"y":2.0},{"x":277.3333333333333,"y":6.0},{"x":280.0,"y":2.0},{"x":281.5,"y":4.0},{"x":283.5,"y":2.0},{"x":285.0,"y":1.0},{"x":286.5,"y":2.0},{"x":288.3333333333333,"y":3.0},{"x":291.5,"y":2.0},{"x":296.0,"y":1.0},{"x":298.0,"y":3.0},{"x":302.0,"y":1.0},{"x":305.5,"y":2.0},{"x":307.3333333333333,"y":3.0},{"x":309.0,"y":2.0},{"x":310.75,"y":4.0},{"x":315.75,"y":4.0},{"x":317.6,"y":5.0},{"x":321.5,"y":4.0},{"x":323.0,"y":1.0},{"x":325.0,"y":2.0},{"x":327.0,"y":3.0},{"x":331.3333333333333,"y":3.0},{"x":333.0,"y":2.0},{"x":335.5,"y":2.0},{"x":338.5,"y":2.0},{"x":341.66666666666663,"y":3.0},{"x":344.3333333333333,"y":3.0},{"x":348.0,"y":5.0},{"x":351.0,"y":1.0},{"x":353.0,"y":2.0},{"x":356.0,"y":1.0},{"x":360.0,"y":1.0},{"x":362.0,"y":1.0},{"x":364.5,"y":2.0},{"x":366.66666666666663,"y":3.0},{"x":368.75,"y":4.0},{"x":373.5,"y":2.0},{"x":375.0,"y":1.0},{"x":377.5,"y":2.0},{"x":379.0,"y":1.0},{"x":382.0,"y":2.0},{"x":384.0,"y":3.0},{"x":386.0,"y":1.0},{"x":389.0,"y":1.0},{"x":392.0,"y":1.0},{"x":393.5,"y":2.0},{"x":395.6,"y":5.0},{"x":397.0,"y":2.0},{"x":399.0,"y":2.0},{"x":400.0,"y":1.0},{"x":401.16666666666663,"y":6.0},{"x":403.40000000000003,"y":5.0},{"x":406.20000000000005,"y":5.0},{"x":409.0,"y":3.0},{"x":411.0,"y":1.0},{"x":413.5,"y":4.0},{"x":417.0,"y":3.0},{"x":418.5,"y":2.0},{"x":421.0,"y":1.0},{"x":424.0,"y":2.0},{"x":427.0,"y":1.0},{"x":429.6,"y":5.0},{"x":431.25,"y":4.0},{"x":435.5,"y":2.0},{"x":437.75,"y":4.0},{"x":439.0,"y":2.0},{"x":443.5,"y":2.0},{"x":446.0,"y":1.0},{"x":448.5,"y":2.0},{"x":452.5,"y":2.0},{"x":454.24999999999994,"y":4.0},{"x":457.66666666666663,"y":3.0},{"x":459.33333333333337,"y":3.0},{"x":462.5,"y":4.0},{"x":466.0,"y":3.0},{"x":467.80000000000007,"y":5.0},{"x":469.16666666666663,"y":6.0},{"x":472.0,"y":1.0},{"x":475.0,"y":1.0},{"x":477.0,"y":1.0},{"x":478.33333333333326,"y":3.0},{"x":480.25,"y":4.0},{"x":482.5,"y":2.0},{"x":484.5,"y":2.0},{"x":487.0,"y":1.0},{"x":489.2,"y":5.0},{"x":491.66666666666663,"y":3.0},{"x":493.0,"y":1.0},{"x":494.5,"y":2.0},{"x":496.0,"y":1.0},{"x":497.75,"y":4.0}] diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 b/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 deleted file mode 100644 index 2953abcf1e644..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 +++ /dev/null @@ -1 +0,0 @@ -0.0 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 b/sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da b/sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f b/sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 493daf5d79c54..0000000000000 --- a/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1 +0,0 @@ -true false true true true false false false true true false true true acc abc abb hive hadoop AaAbAcA false diff --git a/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 b/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 index 398f517b25b5f..2d3e77d99bca3 100644 --- a/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 +++ b/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 @@ -1 +1 @@ -| abc | abc | abc | \ No newline at end of file +| abc | abc | abc | diff --git a/sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c b/sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 b/sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e b/sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index b2890dc4cdb3a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1 +0,0 @@ -0 NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 b/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 index 56a6051ca2b02..d00491fd7e5bb 100644 --- a/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 +++ b/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 @@ -1 +1 @@ -1 \ No newline at end of file +1 diff --git a/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e b/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e index 132dbea91f8a0..5657917e40f7d 100644 --- a/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e +++ b/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e @@ -1 +1 @@ -1 2 2 a 0.1 2 126 128 128 1.0 128 \ No newline at end of file +1 2 2 a 0.1 2 126 128 128 1.0 128 diff --git a/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 b/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 index 51958ff45ab8d..2fcaff3dad9fe 100644 --- a/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 +++ b/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 @@ -1 +1 @@ -1.098612288668 NULL NULL 1.098612288668 NULL NULL 1.584962500721 NULL NULL 0.47712125472 NULL NULL 1.584962500721 NULL NULL NULL -1.0 7.389056098931 8.0 8.0 0.125 8.0 2.0 NaN 1.0 1 8 8 +1.098612288668 NULL NULL 1.098612288668 NULL NULL 1.584962500721 NULL NULL 0.47712125472 NULL NULL 1.584962500721 NULL NULL NULL -1.0 7.389056098931 8.0 8.0 0.125 8.0 2.0 NaN 1.0 1.0 8.0 8.0 diff --git a/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b b/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b index 8a89b039b7151..60b09adaf8029 100644 --- a/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b +++ b/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b @@ -1 +1 @@ -1.0 1.0 1 \ No newline at end of file +1.0 1.0 1 diff --git a/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c b/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c index 83e2e121222a2..cb93f99495494 100644 --- a/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c +++ b/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c @@ -1 +1 @@ --1 2 32 -1 2009-01-01 2009-12-31 2008-03-01 2009-03-02 2008-02-28 2009-02-27 2008-12-31 2008-01-02 2008-02-28 2009-02-27 2006-02-28 2005-02-28 \ No newline at end of file +-1 2 32 -1 2009-01-01 2009-12-31 2008-03-01 2009-03-02 2008-02-28 2009-02-27 2008-12-31 2008-01-02 2008-02-28 2009-02-27 2006-02-28 2005-02-28 diff --git a/sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f b/sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_E-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udf_E-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/udf_E-1-cad0779d18f326c8e453bf2b5fe43596 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b rename to sql/hive/src/test/resources/golden/udf_E-1-cad0779d18f326c8e453bf2b5fe43596 diff --git a/sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c b/sql/hive/src/test/resources/golden/udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c rename to sql/hive/src/test/resources/golden/udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02 diff --git a/sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a b/sql/hive/src/test/resources/golden/udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a rename to sql/hive/src/test/resources/golden/udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a diff --git a/sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f b/sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 b/sql/hive/src/test/resources/golden/udf_E-4-e8924af3bf99d2e01546a965303ffd09 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 rename to sql/hive/src/test/resources/golden/udf_E-4-e8924af3bf99d2e01546a965303ffd09 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 b/sql/hive/src/test/resources/golden/udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 rename to sql/hive/src/test/resources/golden/udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4 diff --git a/sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c b/sql/hive/src/test/resources/golden/udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c rename to sql/hive/src/test/resources/golden/udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02 diff --git a/sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a b/sql/hive/src/test/resources/golden/udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a rename to sql/hive/src/test/resources/golden/udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a diff --git a/sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 b/sql/hive/src/test/resources/golden/udf_E-8-e8924af3bf99d2e01546a965303ffd09 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 rename to sql/hive/src/test/resources/golden/udf_E-8-e8924af3bf99d2e01546a965303ffd09 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f b/sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 b/sql/hive/src/test/resources/golden/udf_PI-1-13fd9345fd15b654d18b2707e5274b2b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 rename to sql/hive/src/test/resources/golden/udf_PI-1-13fd9345fd15b654d18b2707e5274b2b diff --git a/sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 b/sql/hive/src/test/resources/golden/udf_PI-2-97a12f6967726e425469ecfa70177ff0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 rename to sql/hive/src/test/resources/golden/udf_PI-2-97a12f6967726e425469ecfa70177ff0 diff --git a/sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 b/sql/hive/src/test/resources/golden/udf_PI-3-9c1476a2eab7455594e97b338ee3c188 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 rename to sql/hive/src/test/resources/golden/udf_PI-3-9c1476a2eab7455594e97b338ee3c188 diff --git a/sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 b/sql/hive/src/test/resources/golden/udf_PI-4-890f3c276eff2c459d8dc79d5a71c866 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 rename to sql/hive/src/test/resources/golden/udf_PI-4-890f3c276eff2c459d8dc79d5a71c866 diff --git a/sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f b/sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 b/sql/hive/src/test/resources/golden/udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 rename to sql/hive/src/test/resources/golden/udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d diff --git a/sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 b/sql/hive/src/test/resources/golden/udf_PI-6-97a12f6967726e425469ecfa70177ff0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 rename to sql/hive/src/test/resources/golden/udf_PI-6-97a12f6967726e425469ecfa70177ff0 diff --git a/sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 b/sql/hive/src/test/resources/golden/udf_PI-7-9c1476a2eab7455594e97b338ee3c188 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 rename to sql/hive/src/test/resources/golden/udf_PI-7-9c1476a2eab7455594e97b338ee3c188 diff --git a/sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 b/sql/hive/src/test/resources/golden/udf_PI-8-890f3c276eff2c459d8dc79d5a71c866 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 rename to sql/hive/src/test/resources/golden/udf_PI-8-890f3c276eff2c459d8dc79d5a71c866 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_abs-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udf_abs-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 b/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 deleted file mode 100644 index b613b3b9fe967..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 +++ /dev/null @@ -1 +0,0 @@ -abs(x) - returns the absolute value of x \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 b/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 new file mode 100644 index 0000000000000..f1e01bf0d2fc8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 @@ -0,0 +1 @@ +abs(x) - returns the absolute value of x diff --git a/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 b/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 deleted file mode 100644 index c2fb6dc49322d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 +++ /dev/null @@ -1,6 +0,0 @@ -abs(x) - returns the absolute value of x -Example: - > SELECT abs(0) FROM src LIMIT 1; - 0 - > SELECT abs(-5) FROM src LIMIT 1; - 5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 b/sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 b/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 new file mode 100644 index 0000000000000..7dea2445fa616 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 @@ -0,0 +1,6 @@ +abs(x) - returns the absolute value of x +Example: + > SELECT abs(0) FROM src LIMIT 1; + 0 + > SELECT abs(-5) FROM src LIMIT 1; + 5 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 b/sql/hive/src/test/resources/golden/udf_abs-3-50cb3c23902cd29e0dbff188c71062e5 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 rename to sql/hive/src/test/resources/golden/udf_abs-3-50cb3c23902cd29e0dbff188c71062e5 diff --git a/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 b/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 deleted file mode 100644 index f5a78bc6e59b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 +++ /dev/null @@ -1 +0,0 @@ -0 1 123 9223372036854775807 9223372036854775807 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c b/sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 b/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 new file mode 100644 index 0000000000000..c7b1f75a06cb3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 @@ -0,0 +1 @@ +0 1 123 9223372036854775807 9223372036854775807 diff --git a/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 b/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 deleted file mode 100644 index 9e57fa65e8014..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 +++ /dev/null @@ -1 +0,0 @@ -0.0 3.14159265 3.14159265 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/udf_abs-5-5cd4c198e0de884ad436864b95fece6c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 rename to sql/hive/src/test/resources/golden/udf_abs-5-5cd4c198e0de884ad436864b95fece6c diff --git a/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b b/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b new file mode 100644 index 0000000000000..a07dd4e9d970d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b @@ -0,0 +1 @@ +0.0 3.14159265 3.14159265 diff --git a/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 b/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 deleted file mode 100644 index 204db415144a7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 +++ /dev/null @@ -1 +0,0 @@ -acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_acos-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e rename to sql/hive/src/test/resources/golden/udf_acos-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 b/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 new file mode 100644 index 0000000000000..b32b4244d6aeb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 @@ -0,0 +1 @@ +acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise diff --git a/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f b/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f deleted file mode 100644 index d49af1cbf4800..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f +++ /dev/null @@ -1,6 +0,0 @@ -acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise -Example: - > SELECT acos(1) FROM src LIMIT 1; - 0 - > SELECT acos(2) FROM src LIMIT 1; - NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b b/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f b/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f new file mode 100644 index 0000000000000..93925317cd798 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f @@ -0,0 +1,6 @@ +acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise +Example: + > SELECT acos(1) FROM src LIMIT 1; + 0 + > SELECT acos(2) FROM src LIMIT 1; + NULL diff --git a/sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 b/sql/hive/src/test/resources/golden/udf_acos-3-661a0a85283df2a5c1567d60850e362b similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 rename to sql/hive/src/test/resources/golden/udf_acos-3-661a0a85283df2a5c1567d60850e362b diff --git a/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 b/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 deleted file mode 100644 index 5548bdb7cf26a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 +++ /dev/null @@ -1 +0,0 @@ -1.5707963267948966 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf b/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf new file mode 100644 index 0000000000000..53a6e14702ed1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf @@ -0,0 +1 @@ +1.5707963267948966 diff --git a/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a b/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a deleted file mode 100644 index 1f2d6faad9a2c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a +++ /dev/null @@ -1 +0,0 @@ -2.0943951023931957 0.7208187608700897 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e b/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e new file mode 100644 index 0000000000000..edee342d1d902 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e @@ -0,0 +1 @@ +2.0943951023931957 0.7208187608700897 diff --git a/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e b/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 b/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 b/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 index e8f11c444a808..9d794539b4ae0 100644 --- a/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 +++ b/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 @@ -1 +1 @@ -a + b - Returns a+b \ No newline at end of file +a + b - Returns a+b diff --git a/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 b/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 index e8f11c444a808..9d794539b4ae0 100644 --- a/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 +++ b/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 @@ -1 +1 @@ -a + b - Returns a+b \ No newline at end of file +a + b - Returns a+b diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_array-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_array-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee b/sql/hive/src/test/resources/golden/udf_array-1-2e9c2a7d9325bd9a53cf9f181b6333ee similarity index 100% rename from sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee rename to sql/hive/src/test/resources/golden/udf_array-1-2e9c2a7d9325bd9a53cf9f181b6333ee diff --git a/sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a b/sql/hive/src/test/resources/golden/udf_array-2-570741914bb78300b0233e5f38d7f08a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a rename to sql/hive/src/test/resources/golden/udf_array-2-570741914bb78300b0233e5f38d7f08a diff --git a/sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 b/sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc b/sql/hive/src/test/resources/golden/udf_array-3-47818d42e5e7667d8754c3f9a4b8053a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc rename to sql/hive/src/test/resources/golden/udf_array-3-47818d42e5e7667d8754c3f9a4b8053a diff --git a/sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 b/sql/hive/src/test/resources/golden/udf_array-4-51410e4d4d679fe5a8dd7a860f4efc47 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 rename to sql/hive/src/test/resources/golden/udf_array-4-51410e4d4d679fe5a8dd7a860f4efc47 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_array_contains-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udf_array_contains-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a b/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a deleted file mode 100644 index 783e7d086a5c6..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a +++ /dev/null @@ -1 +0,0 @@ -array_contains(array, value) - Returns TRUE if the array contains value. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a b/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a new file mode 100644 index 0000000000000..f17ecea1ab5ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a @@ -0,0 +1 @@ +array_contains(array, value) - Returns TRUE if the array contains value. diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c b/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c deleted file mode 100644 index 0ccae5649af4a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c +++ /dev/null @@ -1,4 +0,0 @@ -array_contains(array, value) - Returns TRUE if the array contains value. -Example: - > SELECT array_contains(array(1, 2, 3), 2) FROM src LIMIT 1; - true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 b/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c b/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c new file mode 100644 index 0000000000000..7f4bbfc04c1c9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c @@ -0,0 +1,4 @@ +array_contains(array, value) - Returns TRUE if the array contains value. +Example: + > SELECT array_contains(array(1, 2, 3), 2) FROM src LIMIT 1; + true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c b/sql/hive/src/test/resources/golden/udf_array_contains-3-6b700cb69af298aef45b96bf5ac862d similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c rename to sql/hive/src/test/resources/golden/udf_array_contains-3-6b700cb69af298aef45b96bf5ac862d diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 b/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d b/sql/hive/src/test/resources/golden/udf_array_contains-4-bdb5a7e0ab81172a438145a1c406b1e8 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d rename to sql/hive/src/test/resources/golden/udf_array_contains-4-bdb5a7e0ab81172a438145a1c406b1e8 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_ascii-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udf_ascii-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c b/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c deleted file mode 100644 index e121c27212c66..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c +++ /dev/null @@ -1 +0,0 @@ -ascii(str) - returns the numeric value of the first character of str \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c b/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c new file mode 100644 index 0000000000000..5e2c4b7209d9b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c @@ -0,0 +1 @@ +ascii(str) - returns the numeric value of the first character of str diff --git a/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 b/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 deleted file mode 100644 index 7c9fb835dfe57..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 +++ /dev/null @@ -1,6 +0,0 @@ -ascii(str) - returns the numeric value of the first character of str -Returns 0 if str is empty or NULL if str is NULL -Example: - > SELECT ascii('222') FROM src LIMIT 1; 50 - > SELECT ascii(2) FROM src LIMIT 1; - 50 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa b/sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 b/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 new file mode 100644 index 0000000000000..87ae6b33499c4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 @@ -0,0 +1,6 @@ +ascii(str) - returns the numeric value of the first character of str +Returns 0 if str is empty or NULL if str is NULL +Example: + > SELECT ascii('222') FROM src LIMIT 1; 50 + > SELECT ascii(2) FROM src LIMIT 1; + 50 diff --git a/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 b/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 deleted file mode 100644 index 726fda4c15ed3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 +++ /dev/null @@ -1 +0,0 @@ -70 0 33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e b/sql/hive/src/test/resources/golden/udf_ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e rename to sql/hive/src/test/resources/golden/udf_ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34 diff --git a/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf b/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf new file mode 100644 index 0000000000000..ba9d9a3b79bab --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf @@ -0,0 +1 @@ +70 0 33 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_asin-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_asin-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd b/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd deleted file mode 100644 index 3a56bffc8da34..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd +++ /dev/null @@ -1 +0,0 @@ -asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 b/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 deleted file mode 100644 index ef207d3f5da38..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 +++ /dev/null @@ -1,6 +0,0 @@ -asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise -Example: - > SELECT asin(0) FROM src LIMIT 1; - 0 - > SELECT asin(2) FROM src LIMIT 1; - NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd b/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd new file mode 100644 index 0000000000000..f5ffb5d0b3939 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd @@ -0,0 +1 @@ +asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise diff --git a/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 b/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 new file mode 100644 index 0000000000000..80c16be805ee8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 @@ -0,0 +1,6 @@ +asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise +Example: + > SELECT asin(0) FROM src LIMIT 1; + 0 + > SELECT asin(2) FROM src LIMIT 1; + NULL diff --git a/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f b/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e b/sql/hive/src/test/resources/golden/udf_asin-3-4b7ee6310a49ebf784a4a712748348ac similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e rename to sql/hive/src/test/resources/golden/udf_asin-3-4b7ee6310a49ebf784a4a712748348ac diff --git a/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 b/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 b/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 deleted file mode 100644 index a9d9b65be4189..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 +++ /dev/null @@ -1 +0,0 @@ --0.5235987755982989 0.7208187608700897 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce b/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 b/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 b/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 new file mode 100644 index 0000000000000..46381bfe173d5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 @@ -0,0 +1 @@ +-0.5235987755982989 0.7208187608700897 diff --git a/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f b/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_atan-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udf_atan-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b deleted file mode 100644 index 019676ad50bc3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b +++ /dev/null @@ -1 +0,0 @@ -atan(x) - returns the atan (arctan) of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 deleted file mode 100644 index ce9e52716ff44..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 +++ /dev/null @@ -1,4 +0,0 @@ -atan(x) - returns the atan (arctan) of x (x is in radians) -Example: - > SELECT atan(0) FROM src LIMIT 1; - 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b new file mode 100644 index 0000000000000..1d34577c5c6b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b @@ -0,0 +1 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 new file mode 100644 index 0000000000000..6d7f1d49d9e72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 @@ -0,0 +1,4 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) +Example: + > SELECT atan(0) FROM src LIMIT 1; + 0 diff --git a/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 b/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 b/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 deleted file mode 100644 index 37b3e6018d62c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 +++ /dev/null @@ -1 +0,0 @@ -0.7853981633974483 1.4056476493802699 -0.7853981633974483 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e b/sql/hive/src/test/resources/golden/udf_atan-3-e6f97a834028a67e6c3033949f98fbf8 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e rename to sql/hive/src/test/resources/golden/udf_atan-3-e6f97a834028a67e6c3033949f98fbf8 diff --git a/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b deleted file mode 100644 index 019676ad50bc3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b +++ /dev/null @@ -1 +0,0 @@ -atan(x) - returns the atan (arctan) of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 b/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 new file mode 100644 index 0000000000000..fe43015585aa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 @@ -0,0 +1 @@ +0.7853981633974483 1.4056476493802699 -0.7853981633974483 diff --git a/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 deleted file mode 100644 index ce9e52716ff44..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 +++ /dev/null @@ -1,4 +0,0 @@ -atan(x) - returns the atan (arctan) of x (x is in radians) -Example: - > SELECT atan(0) FROM src LIMIT 1; - 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b new file mode 100644 index 0000000000000..1d34577c5c6b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b @@ -0,0 +1 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 new file mode 100644 index 0000000000000..6d7f1d49d9e72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 @@ -0,0 +1,4 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) +Example: + > SELECT atan(0) FROM src LIMIT 1; + 0 diff --git a/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 b/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 b/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 deleted file mode 100644 index 37b3e6018d62c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 +++ /dev/null @@ -1 +0,0 @@ -0.7853981633974483 1.4056476493802699 -0.7853981633974483 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d b/sql/hive/src/test/resources/golden/udf_atan-7-e6f97a834028a67e6c3033949f98fbf8 similarity index 100% rename from sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d rename to sql/hive/src/test/resources/golden/udf_atan-7-e6f97a834028a67e6c3033949f98fbf8 diff --git a/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 b/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 new file mode 100644 index 0000000000000..fe43015585aa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 @@ -0,0 +1 @@ +0.7853981633974483 1.4056476493802699 -0.7853981633974483 diff --git a/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 b/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 index bd171b0e9a29c..13ba346a52c93 100644 --- a/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 +++ b/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 @@ -1 +1 @@ -avg(x) - Returns the mean of a set of numbers \ No newline at end of file +avg(x) - Returns the mean of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f b/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f index bd171b0e9a29c..13ba346a52c93 100644 --- a/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f +++ b/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f @@ -1 +1 @@ -avg(x) - Returns the mean of a set of numbers \ No newline at end of file +avg(x) - Returns the mean of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 b/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 index 27b6cd42b68cf..c538e81220b1f 100644 --- a/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 +++ b/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 @@ -1 +1 @@ -There is no documentation for function 'bigint' \ No newline at end of file +There is no documentation for function 'bigint' diff --git a/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 b/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 index 27b6cd42b68cf..c538e81220b1f 100644 --- a/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 +++ b/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 @@ -1 +1 @@ -There is no documentation for function 'bigint' \ No newline at end of file +There is no documentation for function 'bigint' diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_bin-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udf_bin-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b b/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b deleted file mode 100644 index 4790a2c1f9292..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b +++ /dev/null @@ -1 +0,0 @@ -bin(n) - returns n in binary \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 b/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 deleted file mode 100644 index 4d82712e858e4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 +++ /dev/null @@ -1,5 +0,0 @@ -bin(n) - returns n in binary -n is a BIGINT. Returns NULL if n is NULL. -Example: - > SELECT bin(13) FROM src LIMIT 1 - '1101' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b b/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b new file mode 100644 index 0000000000000..ef60184c54fb5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b @@ -0,0 +1 @@ +bin(n) - returns n in binary diff --git a/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b b/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b deleted file mode 100644 index 20fae133f0b4d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b +++ /dev/null @@ -1 +0,0 @@ -1 0 101111101011100001101100101 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 b/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 new file mode 100644 index 0000000000000..115f93fe0fd3e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 @@ -0,0 +1,5 @@ +bin(n) - returns n in binary +n is a BIGINT. Returns NULL if n is NULL. +Example: + > SELECT bin(13) FROM src LIMIT 1 + '1101' diff --git a/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab b/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab new file mode 100644 index 0000000000000..83b020539cb9a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab @@ -0,0 +1 @@ +1 0 101111101011100001101100101 diff --git a/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 b/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 deleted file mode 100644 index 76ea1be7f9214..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 +++ /dev/null @@ -1 +0,0 @@ -1111111111111111111111111111111111111111111111111111111111111011 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 b/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 new file mode 100644 index 0000000000000..1b12d2f6de475 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 @@ -0,0 +1 @@ +1111111111111111111111111111111111111111111111111111111111111011 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_bitmap_and-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_bitmap_and-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d b/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d deleted file mode 100644 index 652a71b48011c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d +++ /dev/null @@ -1 +0,0 @@ -[13,2,4,8589934592,4096,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 deleted file mode 100644 index 6d061bf1198c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 +++ /dev/null @@ -1 +0,0 @@ -[13,1,4,2,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 new file mode 100644 index 0000000000000..1211192ad4be3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 @@ -0,0 +1 @@ +[13,2,4,8589934592,4096,0] diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 new file mode 100644 index 0000000000000..33b9ad33d4f01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 @@ -0,0 +1 @@ +[13,1,4,2,0] diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 b/sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 b/sql/hive/src/test/resources/golden/udf_bitmap_and-3-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 rename to sql/hive/src/test/resources/golden/udf_bitmap_and-3-d550d017f9eb9176593719ea34eaae9b diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 b/sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 b/sql/hive/src/test/resources/golden/udf_bitmap_and-4-a486db1f5a06f9623a0e5abd7737b0c6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 rename to sql/hive/src/test/resources/golden/udf_bitmap_and-4-a486db1f5a06f9623a0e5abd7737b0c6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 b/sql/hive/src/test/resources/golden/udf_bitmap_and-5-549dbeb1293c4c49ae08bf08acdbdf23 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 rename to sql/hive/src/test/resources/golden/udf_bitmap_and-5-549dbeb1293c4c49ae08bf08acdbdf23 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 b/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 deleted file mode 100644 index 43be09952b09c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 +++ /dev/null @@ -1,10 +0,0 @@ -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 new file mode 100644 index 0000000000000..0db773930c274 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 @@ -0,0 +1,10 @@ +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/udf_bitmap_and-7-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 rename to sql/hive/src/test/resources/golden/udf_bitmap_and-7-d550d017f9eb9176593719ea34eaae9b diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 b/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df b/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df deleted file mode 100644 index 02e4a84d62c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df +++ /dev/null @@ -1 +0,0 @@ -false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad b/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a174269b5d1757398ab7f89cf1c97bfa similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a174269b5d1757398ab7f89cf1c97bfa diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 b/sql/hive/src/test/resources/golden/udf_bitmap_empty-2-f5d9880a3278b5632b356bbe6ecd90d3 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-2-f5d9880a3278b5632b356bbe6ecd90d3 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c b/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c deleted file mode 100644 index 652a71b48011c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c +++ /dev/null @@ -1 +0,0 @@ -[13,2,4,8589934592,4096,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_bitmap_or-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udf_bitmap_or-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef deleted file mode 100644 index 163f4734f4c21..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef +++ /dev/null @@ -1 +0,0 @@ -[13,2,4,8589934592,4224,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef new file mode 100644 index 0000000000000..1211192ad4be3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef @@ -0,0 +1 @@ +[13,2,4,8589934592,4096,0] diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 new file mode 100644 index 0000000000000..c1bd6bef82398 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 @@ -0,0 +1 @@ +[13,2,4,8589934592,4224,0] diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 b/sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d b/sql/hive/src/test/resources/golden/udf_bitmap_or-3-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d rename to sql/hive/src/test/resources/golden/udf_bitmap_or-3-d550d017f9eb9176593719ea34eaae9b diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 b/sql/hive/src/test/resources/golden/udf_bitmap_or-4-a486db1f5a06f9623a0e5abd7737b0c6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 rename to sql/hive/src/test/resources/golden/udf_bitmap_or-4-a486db1f5a06f9623a0e5abd7737b0c6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 b/sql/hive/src/test/resources/golden/udf_bitmap_or-5-549dbeb1293c4c49ae08bf08acdbdf23 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 rename to sql/hive/src/test/resources/golden/udf_bitmap_or-5-549dbeb1293c4c49ae08bf08acdbdf23 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 deleted file mode 100644 index c977a4ce25c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 +++ /dev/null @@ -1,10 +0,0 @@ -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 new file mode 100644 index 0000000000000..62f4e245e2cdd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 @@ -0,0 +1,10 @@ +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 b/sql/hive/src/test/resources/golden/udf_bitmap_or-7-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 rename to sql/hive/src/test/resources/golden/udf_bitmap_or-7-d550d017f9eb9176593719ea34eaae9b diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a b/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a index 71cadba7649e8..bb00e0889dc07 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a +++ b/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a @@ -1 +1 @@ -a & b - Bitwise and \ No newline at end of file +a & b - Bitwise and diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b b/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b index ead02cba53aa4..dc67d624d14be 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b +++ b/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b @@ -1,4 +1,4 @@ a & b - Bitwise and Example: > SELECT 3 & 5 FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f b/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f index 8b06e374c708d..d417f76be9940 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f +++ b/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f @@ -1 +1 @@ -~ n - Bitwise not \ No newline at end of file +~ n - Bitwise not diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d b/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d index 413ca8a653e1f..db249e9032680 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d +++ b/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d @@ -1,4 +1,4 @@ ~ n - Bitwise not Example: > SELECT ~ 0 FROM src LIMIT 1; - -1 \ No newline at end of file + -1 diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 b/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 index bdbd28cad97c4..32343b4afe8bb 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 +++ b/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 @@ -1 +1 @@ -a | b - Bitwise or \ No newline at end of file +a | b - Bitwise or diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb b/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb index 4c116be2acae3..8ced93b4f8e88 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb +++ b/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb @@ -1,4 +1,4 @@ a | b - Bitwise or Example: > SELECT 3 | 5 FROM src LIMIT 1; - 7 \ No newline at end of file + 7 diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 b/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 index b483f75f1b956..778fda3969951 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 +++ b/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 @@ -1 +1 @@ -a ^ b - Bitwise exclusive or \ No newline at end of file +a ^ b - Bitwise exclusive or diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f index 03bf261d7cf51..6f1ab55c41ce7 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f +++ b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f @@ -1,4 +1,4 @@ a ^ b - Bitwise exclusive or Example: > SELECT 3 ^ 5 FROM src LIMIT 1; - 2 \ No newline at end of file + 2 diff --git a/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 b/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 index 8134698dfbdcd..495b36e68ecde 100644 --- a/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 +++ b/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 @@ -1 +1 @@ -There is no documentation for function 'boolean' \ No newline at end of file +There is no documentation for function 'boolean' diff --git a/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e b/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e index 8134698dfbdcd..495b36e68ecde 100644 --- a/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e +++ b/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e @@ -1 +1 @@ -There is no documentation for function 'boolean' \ No newline at end of file +There is no documentation for function 'boolean' diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udf_case-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 rename to sql/hive/src/test/resources/golden/udf_case-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 b/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 deleted file mode 100644 index 645bd7302810f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 +++ /dev/null @@ -1 +0,0 @@ -There is no documentation for function 'case' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 b/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 deleted file mode 100644 index 645bd7302810f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 +++ /dev/null @@ -1 +0,0 @@ -There is no documentation for function 'case' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 b/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 new file mode 100644 index 0000000000000..e5bc5948a0894 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 @@ -0,0 +1 @@ +There is no documentation for function 'case' diff --git a/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 b/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 new file mode 100644 index 0000000000000..e5bc5948a0894 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 @@ -0,0 +1 @@ +There is no documentation for function 'case' diff --git a/sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f b/sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 b/sql/hive/src/test/resources/golden/udf_case-3-48789112d79aeb450d9f49184fc20e1c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 rename to sql/hive/src/test/resources/golden/udf_case-3-48789112d79aeb450d9f49184fc20e1c diff --git a/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c b/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c deleted file mode 100644 index 1ca086d1dca4d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c +++ /dev/null @@ -1 +0,0 @@ -2 5 15 NULL 20 24 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 b/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 deleted file mode 100644 index b920295f69a53..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 +++ /dev/null @@ -1 +0,0 @@ -yo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 b/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 new file mode 100644 index 0000000000000..a7f5e19894027 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 @@ -0,0 +1 @@ +2 5 15 NULL 20 24 diff --git a/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb b/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb new file mode 100644 index 0000000000000..092bfb9bdf74d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb @@ -0,0 +1 @@ +yo diff --git a/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc b/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc new file mode 100644 index 0000000000000..db46b5a72aa80 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc @@ -0,0 +1 @@ +123 123.0 abcd diff --git a/sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c b/sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 b/sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 b/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 deleted file mode 100644 index d97a0ef52b2e8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 +++ /dev/null @@ -1,3 +0,0 @@ -1 zero 0 -3 10 is ten NULL -100 default NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 b/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 index 4854a3c7773ac..badaea42f1d2c 100644 --- a/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 +++ b/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 @@ -1 +1 @@ -ceil(x) - Find the smallest integer not smaller than x \ No newline at end of file +ceil(x) - Find the smallest integer not smaller than x diff --git a/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb b/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb index 23685df97892d..7d608c5ebd2a3 100644 --- a/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb +++ b/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb @@ -4,4 +4,4 @@ Example: > SELECT ceil(-0.1) FROM src LIMIT 1; 0 > SELECT ceil(5) FROM src LIMIT 1; - 5 \ No newline at end of file + 5 diff --git a/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd b/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd index bf61b7e89d8b3..31d18aaefac89 100644 --- a/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd +++ b/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd @@ -1 +1 @@ -ceiling(x) - Find the smallest integer not smaller than x \ No newline at end of file +ceiling(x) - Find the smallest integer not smaller than x diff --git a/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df b/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df index 79a06f0ee2a5d..564d0566997d4 100644 --- a/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df +++ b/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df @@ -4,4 +4,4 @@ Example: > SELECT ceiling(-0.1) FROM src LIMIT 1; 0 > SELECT ceiling(5) FROM src LIMIT 1; - 5 \ No newline at end of file + 5 diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 b/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 deleted file mode 100644 index 66313c54cc684..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 +++ /dev/null @@ -1 +0,0 @@ -coalesce(a1, a2, ...) - Returns the first non-null argument \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 b/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 deleted file mode 100644 index 04d69977c3df4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 +++ /dev/null @@ -1,4 +0,0 @@ -coalesce(a1, a2, ...) - Returns the first non-null argument -Example: - > SELECT coalesce(NULL, 1, NULL) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 b/sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 b/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 deleted file mode 100644 index cd5686d89eba4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 +++ /dev/null @@ -1 +0,0 @@ -1 1 2 1 3 4 1 1 2 1 3 4 1.0 1.0 2.0 2.0 2.0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 b/sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 b/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 deleted file mode 100644 index 98339f97ef2f8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 +++ /dev/null @@ -1,11 +0,0 @@ -0 0 999 -2 1 999 -4 8 value_2 -6 27 999 -8 64 999 -10 125 999 -12 216 999 -14 343 999 -16 512 999 -18 729 999 -999 999 999 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 b/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 deleted file mode 100644 index 59d3f01051903..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_udf_get_java_string 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaString') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_concat-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to sql/hive/src/test/resources/golden/udf_concat-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 deleted file mode 100644 index a3998eeeee623..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 +++ /dev/null @@ -1 +0,0 @@ -concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 b/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 deleted file mode 100644 index 0afd542a05033..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 +++ /dev/null @@ -1,5 +0,0 @@ -concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN -Returns NULL if any argument is NULL. -Example: - > SELECT concat('abc', 'def') FROM src LIMIT 1; - 'abcdef' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 new file mode 100644 index 0000000000000..b576089faa484 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 @@ -0,0 +1 @@ +concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN diff --git a/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 b/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 deleted file mode 100644 index 69af4e63792dc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 +++ /dev/null @@ -1 +0,0 @@ -ab abc NULL NULL a NULL 123a 12 1 1234abcextra argument \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 b/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 new file mode 100644 index 0000000000000..e303d18a08100 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 @@ -0,0 +1,5 @@ +concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN +Returns NULL if any argument is NULL. +Example: + > SELECT concat('abc', 'def') FROM src LIMIT 1; + 'abcdef' diff --git a/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 b/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 new file mode 100644 index 0000000000000..21f03c7abb9b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 @@ -0,0 +1 @@ +ab abc NULL NULL a NULL 123a 12 1 1234abcextra argument diff --git a/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde b/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde new file mode 100644 index 0000000000000..09b34f2aa1b69 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde @@ -0,0 +1 @@ +abcd abcd diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f index aecd5d9f2a5c2..f4cd225e4b2b7 100644 --- a/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f @@ -54,4 +54,4 @@ 1234 95 1234 96 1234 97 -1234 98 \ No newline at end of file +1234 98 diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f index 455cef3f57162..fcdbdaa75fd52 100644 --- a/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 1234abcextra argument val_28 1234abcextra argument val_37 1234abcextra argument val_90 -1234abcextra argument val_97 \ No newline at end of file +1234abcextra argument val_97 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 b/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 deleted file mode 100644 index c901f70500cba..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 +++ /dev/null @@ -1 +0,0 @@ -concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_concat_ws-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/udf_concat_ws-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 b/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 new file mode 100644 index 0000000000000..14d190e03b755 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 @@ -0,0 +1 @@ +concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator. diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab b/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab deleted file mode 100644 index 3e751463e055d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab +++ /dev/null @@ -1,4 +0,0 @@ -concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator. -Example: - > SELECT concat_ws('.', 'www', array('facebook', 'com')) FROM src LIMIT 1; - 'www.facebook.com' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab b/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab new file mode 100644 index 0000000000000..f3be71e54421c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab @@ -0,0 +1,4 @@ +concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator. +Example: + > SELECT concat_ws('.', 'www', array('facebook', 'com')) FROM src LIMIT 1; + 'www.facebook.com' diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c b/sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 b/sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e b/sql/hive/src/test/resources/golden/udf_concat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e rename to sql/hive/src/test/resources/golden/udf_concat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f b/sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 b/sql/hive/src/test/resources/golden/udf_concat_ws-4-b13a1f7f63d2a54efa331c82bd635d63 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 rename to sql/hive/src/test/resources/golden/udf_concat_ws-4-b13a1f7f63d2a54efa331c82bd635d63 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 b/sql/hive/src/test/resources/golden/udf_concat_ws-5-a507af4996b13433b0ae100fcb32358f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 rename to sql/hive/src/test/resources/golden/udf_concat_ws-5-a507af4996b13433b0ae100fcb32358f diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 b/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 deleted file mode 100644 index 7896fd787f3a2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 +++ /dev/null @@ -1 +0,0 @@ -xyzabc8675309 abc,xyz,8675309 NULL abc**8675309 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 b/sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 b/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 new file mode 100644 index 0000000000000..720cafd9370a7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 @@ -0,0 +1 @@ +xyzabc8675309 abc,xyz,8675309 NULL abc**8675309 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c b/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c deleted file mode 100644 index a0ec688a3b084..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c +++ /dev/null @@ -1 +0,0 @@ -www.face.book.com.1234 www-face-book-com-1234 wwwFfaceFbookFcomF1234 www_face_book_com_1234 www**face**book**com**1234 www[]face[]book[]com[]1234 wwwAAAfaceAAAbookAAAcomAAA1234 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c b/sql/hive/src/test/resources/golden/udf_concat_ws-7-97071809ba7701b78e3729996f14b591 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c rename to sql/hive/src/test/resources/golden/udf_concat_ws-7-97071809ba7701b78e3729996f14b591 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 b/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 new file mode 100644 index 0000000000000..93b36d28322c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 @@ -0,0 +1 @@ +www.face.book.com.1234 www-face-book-com-1234 wwwFfaceFbookFcomF1234 www_face_book_com_1234 www**face**book**com**1234 www[]face[]book[]com[]1234 wwwAAAfaceAAAbookAAAcomAAA1234 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 b/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 deleted file mode 100644 index 0c9f2d12ba117..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 b/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 new file mode 100644 index 0000000000000..edb4b1f84001b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 @@ -0,0 +1 @@ +NULL NULL NULL NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_conv-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to sql/hive/src/test/resources/golden/udf_conv-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a b/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a deleted file mode 100644 index b8fbe88a19971..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a +++ /dev/null @@ -1 +0,0 @@ -conv(num, from_base, to_base) - convert num from from_base to to_base \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab b/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab deleted file mode 100644 index 4f4b0c594c459..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab +++ /dev/null @@ -1,7 +0,0 @@ -conv(num, from_base, to_base) - convert num from from_base to to_base -If to_base is negative, treat num as a signed integer,otherwise, treat it as an unsigned integer. -Example: - > SELECT conv('100', 2, 10) FROM src LIMIT 1; - '4' - > SELECT conv(-10, 16, -10) FROM src LIMIT 1; - '16' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a b/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a new file mode 100644 index 0000000000000..0753228c31bd6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a @@ -0,0 +1 @@ +conv(num, from_base, to_base) - convert num from from_base to to_base diff --git a/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 b/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 deleted file mode 100644 index 9512cc4241554..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 +++ /dev/null @@ -1,3 +0,0 @@ -EE 568 -56 134 -137 785 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 b/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 new file mode 100644 index 0000000000000..191900972dd95 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 @@ -0,0 +1 @@ +5 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f b/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f new file mode 100644 index 0000000000000..ad3fa0267fa6b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f @@ -0,0 +1,3 @@ +EE 568 +56 134 +137 785 diff --git a/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab b/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab new file mode 100644 index 0000000000000..2d66703d6f3f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab @@ -0,0 +1,7 @@ +conv(num, from_base, to_base) - convert num from from_base to to_base +If to_base is negative, treat num as a signed integer,otherwise, treat it as an unsigned integer. +Example: + > SELECT conv('100', 2, 10) FROM src LIMIT 1; + '4' + > SELECT conv(-10, 16, -10) FROM src LIMIT 1; + '16' diff --git a/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 b/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 deleted file mode 100644 index 91e56a11fa628..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 +++ /dev/null @@ -1 +0,0 @@ -3HL 22 33 116ED2B2FB4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a b/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a new file mode 100644 index 0000000000000..4563fcc478648 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a @@ -0,0 +1 @@ +3HL 22 33 116ED2B2FB4 diff --git a/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d b/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d deleted file mode 100644 index 08f70d742b8e4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d +++ /dev/null @@ -1 +0,0 @@ --641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 b/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 deleted file mode 100644 index a2a44daa718cc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 +++ /dev/null @@ -1 +0,0 @@ -FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a b/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a new file mode 100644 index 0000000000000..632c3705a0493 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a @@ -0,0 +1 @@ +-641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 diff --git a/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 b/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 deleted file mode 100644 index 6948fe3783119..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 +++ /dev/null @@ -1 +0,0 @@ -5 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 b/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 new file mode 100644 index 0000000000000..3a7a2ec34f909 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 diff --git a/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a b/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a deleted file mode 100644 index e1021e50fdcbb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a +++ /dev/null @@ -1 +0,0 @@ -3HL 22 33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f b/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f new file mode 100644 index 0000000000000..191900972dd95 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f @@ -0,0 +1 @@ +5 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b b/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b new file mode 100644 index 0000000000000..c5348e173c243 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b @@ -0,0 +1 @@ +3HL 22 33 diff --git a/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e b/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e deleted file mode 100644 index 08f70d742b8e4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e +++ /dev/null @@ -1 +0,0 @@ --641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb b/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb deleted file mode 100644 index a2a44daa718cc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb +++ /dev/null @@ -1 +0,0 @@ -FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 b/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 new file mode 100644 index 0000000000000..632c3705a0493 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 @@ -0,0 +1 @@ +-641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 diff --git a/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 b/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 new file mode 100644 index 0000000000000..3a7a2ec34f909 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 diff --git a/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 b/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 deleted file mode 100644 index 6948fe3783119..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 +++ /dev/null @@ -1 +0,0 @@ -5 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 b/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 deleted file mode 100644 index 99a7d42bb5b57..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 +++ /dev/null @@ -1 +0,0 @@ -cos(x) - returns the cosine of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_cos-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to sql/hive/src/test/resources/golden/udf_cos-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 b/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 deleted file mode 100644 index 196294de1f19f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 +++ /dev/null @@ -1,4 +0,0 @@ -cos(x) - returns the cosine of x (x is in radians) -Example: - > SELECT cos(0) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 b/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 new file mode 100644 index 0000000000000..fa0e6975503cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 @@ -0,0 +1 @@ +cos(x) - returns the cosine of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 b/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 new file mode 100644 index 0000000000000..09d6150733802 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 @@ -0,0 +1,4 @@ +cos(x) - returns the cosine of x (x is in radians) +Example: + > SELECT cos(0) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 b/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 b/sql/hive/src/test/resources/golden/udf_cos-3-166acc86afd6ececfe43800e38f106c9 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 rename to sql/hive/src/test/resources/golden/udf_cos-3-166acc86afd6ececfe43800e38f106c9 diff --git a/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 b/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 deleted file mode 100644 index c0690acff887b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 +++ /dev/null @@ -1 +0,0 @@ -0.5570225467662173 7.963267107332633E-4 0.8775825618903728 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 b/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 new file mode 100644 index 0000000000000..14417ab71b694 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 @@ -0,0 +1 @@ +0.5570225467662173 7.963267107332633E-4 0.8775825618903728 diff --git a/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 b/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 index e01715295adc8..56cf7bdf7b040 100644 --- a/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 +++ b/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 @@ -1,3 +1,3 @@ count(*) - Returns the total number of retrieved rows, including rows containing NULL values. count(expr) - Returns the number of rows for which the supplied expression is non-NULL. -count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL. \ No newline at end of file +count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL. diff --git a/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f b/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f index e01715295adc8..56cf7bdf7b040 100644 --- a/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f +++ b/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f @@ -1,3 +1,3 @@ count(*) - Returns the total number of retrieved rows, including rows containing NULL values. count(expr) - Returns the number of rows for which the supplied expression is non-NULL. -count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL. \ No newline at end of file +count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL. diff --git a/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e b/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e +++ b/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e b/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e +++ b/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 b/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 +++ b/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 b/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 +++ b/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 b/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 index e0da8ae09ae60..7536e3d326192 100644 --- a/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 +++ b/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 @@ -1 +1 @@ -309 \ No newline at end of file +309 diff --git a/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 b/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 index e0da8ae09ae60..7536e3d326192 100644 --- a/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 +++ b/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 @@ -1 +1 @@ -309 \ No newline at end of file +309 diff --git a/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 b/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 +++ b/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 b/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 index 83e0fc4e63a25..2e77bafd12f7d 100644 --- a/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 +++ b/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 @@ -1 +1 @@ -date_add(start_date, num_days) - Returns the date that is num_days after start_date. \ No newline at end of file +date_add(start_date, num_days) - Returns the date that is num_days after start_date. diff --git a/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 b/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 index 83b9851499f71..3c91e138d7bd5 100644 --- a/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 +++ b/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 @@ -2,4 +2,4 @@ date_add(start_date, num_days) - Returns the date that is num_days after start_d start_date is a string in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. num_days is a number. The time part of start_date is ignored. Example: > SELECT date_add('2009-30-07', 1) FROM src LIMIT 1; - '2009-31-07' \ No newline at end of file + '2009-31-07' diff --git a/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 b/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 index 7dec81f2b6d4e..3ee6ac4820852 100644 --- a/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 +++ b/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 @@ -1 +1 @@ -date_sub(start_date, num_days) - Returns the date that is num_days before start_date. \ No newline at end of file +date_sub(start_date, num_days) - Returns the date that is num_days before start_date. diff --git a/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 b/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 index 105b63424062b..29d663f35c586 100644 --- a/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 +++ b/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 @@ -2,4 +2,4 @@ date_sub(start_date, num_days) - Returns the date that is num_days before start_ start_date is a string in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. num_days is a number. The time part of start_date is ignored. Example: > SELECT date_sub('2009-30-07', 1) FROM src LIMIT 1; - '2009-29-07' \ No newline at end of file + '2009-29-07' diff --git a/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 b/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 index b59d456397d53..64cae647c8005 100644 --- a/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 +++ b/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 @@ -1 +1 @@ -datediff(date1, date2) - Returns the number of days between date1 and date2 \ No newline at end of file +datediff(date1, date2) - Returns the number of days between date1 and date2 diff --git a/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 b/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 index c240df94594d4..7ccaee7ad3bd4 100644 --- a/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 +++ b/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 @@ -2,4 +2,4 @@ datediff(date1, date2) - Returns the number of days between date1 and date2 date1 and date2 are strings in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. The time parts are ignored.If date1 is earlier than date2, the result is negative. Example: > SELECT datediff('2009-30-07', '2009-31-07') FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 b/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 index 11e32a4509b3f..d4017178b4e6b 100644 --- a/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 +++ b/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 @@ -1 +1 @@ -day(date) - Returns the date of the month of date \ No newline at end of file +day(date) - Returns the date of the month of date diff --git a/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 b/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 index 9da0858ba92a5..6135aafa50860 100644 --- a/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 +++ b/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 @@ -3,4 +3,4 @@ Synonyms: dayofmonth date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. Example: > SELECT day('2009-30-07', 1) FROM src LIMIT 1; - 30 \ No newline at end of file + 30 diff --git a/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 b/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 index 33e90a2af295f..47a7018d9d5ac 100644 --- a/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 +++ b/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 @@ -1 +1 @@ -dayofmonth(date) - Returns the date of the month of date \ No newline at end of file +dayofmonth(date) - Returns the date of the month of date diff --git a/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 b/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 index ee9911af3248c..d9490e20a3b6d 100644 --- a/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 +++ b/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 @@ -3,4 +3,4 @@ Synonyms: day date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. Example: > SELECT dayofmonth('2009-30-07', 1) FROM src LIMIT 1; - 30 \ No newline at end of file + 30 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_degrees-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/udf_degrees-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae b/sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_degrees-1-f24ce67606944e23a4adc79f91cf0c17 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_degrees-1-f24ce67606944e23a4adc79f91cf0c17 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 b/sql/hive/src/test/resources/golden/udf_degrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 rename to sql/hive/src/test/resources/golden/udf_degrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b b/sql/hive/src/test/resources/golden/udf_degrees-3-2fd3a55901fe765f8f154531a7f5fd6b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b rename to sql/hive/src/test/resources/golden/udf_degrees-3-2fd3a55901fe765f8f154531a7f5fd6b diff --git a/sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/sql/hive/src/test/resources/golden/udf_degrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 rename to sql/hive/src/test/resources/golden/udf_degrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae b/sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_degrees-5-3a6468b02be2605c91b31987e76fb9a8 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_degrees-5-3a6468b02be2605c91b31987e76fb9a8 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 b/sql/hive/src/test/resources/golden/udf_degrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 rename to sql/hive/src/test/resources/golden/udf_degrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b b/sql/hive/src/test/resources/golden/udf_degrees-7-2fd3a55901fe765f8f154531a7f5fd6b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b rename to sql/hive/src/test/resources/golden/udf_degrees-7-2fd3a55901fe765f8f154531a7f5fd6b diff --git a/sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/sql/hive/src/test/resources/golden/udf_degrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 rename to sql/hive/src/test/resources/golden/udf_degrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96 diff --git a/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db b/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db deleted file mode 100644 index d98718752a36f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db +++ /dev/null @@ -1 +0,0 @@ -a div b - Divide a by b rounded to the long integer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/sql/hive/src/test/resources/golden/udf_div-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e rename to sql/hive/src/test/resources/golden/udf_div-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db b/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db new file mode 100644 index 0000000000000..b02a7003ce768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db @@ -0,0 +1 @@ +a div b - Divide a by b rounded to the long integer diff --git a/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 b/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 deleted file mode 100644 index 59265a74f9b3f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 +++ /dev/null @@ -1,4 +0,0 @@ -a div b - Divide a by b rounded to the long integer -Example: - > SELECT 3 div 2 FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b b/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 b/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 new file mode 100644 index 0000000000000..6b79eff6e6092 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 @@ -0,0 +1,4 @@ +a div b - Divide a by b rounded to the long integer +Example: + > SELECT 3 div 2 FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 b/sql/hive/src/test/resources/golden/udf_div-3-5111340caad64e36370d9d4bc4db5f27 similarity index 100% rename from sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 rename to sql/hive/src/test/resources/golden/udf_div-3-5111340caad64e36370d9d4bc4db5f27 diff --git a/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 b/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 deleted file mode 100644 index 8b623e47785f6..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 +++ /dev/null @@ -1 +0,0 @@ -a / b - Divide a by b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 b/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 deleted file mode 100644 index 2acf2b6a64648..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 +++ /dev/null @@ -1,4 +0,0 @@ -a / b - Divide a by b -Example: - > SELECT 3 / 2 FROM src LIMIT 1; - 1.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 b/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 deleted file mode 100644 index 400122e60f599..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 +++ /dev/null @@ -1 +0,0 @@ -1.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 b/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 index 54de9e9f8edec..1d0a61f1b635f 100644 --- a/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 +++ b/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 @@ -1 +1 @@ -There is no documentation for function 'double' \ No newline at end of file +There is no documentation for function 'double' diff --git a/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 b/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 index 54de9e9f8edec..1d0a61f1b635f 100644 --- a/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 +++ b/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 @@ -1 +1 @@ -There is no documentation for function 'double' \ No newline at end of file +There is no documentation for function 'double' diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 b/sql/hive/src/test/resources/golden/udf_elt-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 rename to sql/hive/src/test/resources/golden/udf_elt-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f b/sql/hive/src/test/resources/golden/udf_elt-1-b46b060da76d1772db998c26a62a608f similarity index 100% rename from sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f rename to sql/hive/src/test/resources/golden/udf_elt-1-b46b060da76d1772db998c26a62a608f diff --git a/sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a b/sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 b/sql/hive/src/test/resources/golden/udf_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 rename to sql/hive/src/test/resources/golden/udf_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_elt-3-c2554fac72a2a51bb33faae16aec3507 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_elt-3-c2554fac72a2a51bb33faae16aec3507 diff --git a/sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 b/sql/hive/src/test/resources/golden/udf_elt-4-533ad9c703c320a6556c09dd5f9ac351 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 rename to sql/hive/src/test/resources/golden/udf_elt-4-533ad9c703c320a6556c09dd5f9ac351 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/udf_equal-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 rename to sql/hive/src/test/resources/golden/udf_equal-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 b/sql/hive/src/test/resources/golden/udf_equal-1-36b6cdf7c5f68c91155569b1622f5876 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 rename to sql/hive/src/test/resources/golden/udf_equal-1-36b6cdf7c5f68c91155569b1622f5876 diff --git a/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 b/sql/hive/src/test/resources/golden/udf_equal-2-2422b50b96502dde8b661acdfebd8892 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 rename to sql/hive/src/test/resources/golden/udf_equal-2-2422b50b96502dde8b661acdfebd8892 diff --git a/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 b/sql/hive/src/test/resources/golden/udf_equal-3-e0faab0f5e736c24bcc5503aeac55053 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 rename to sql/hive/src/test/resources/golden/udf_equal-3-e0faab0f5e736c24bcc5503aeac55053 diff --git a/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 b/sql/hive/src/test/resources/golden/udf_equal-4-39d8d6f197803de927f0af5409ec2f33 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 rename to sql/hive/src/test/resources/golden/udf_equal-4-39d8d6f197803de927f0af5409ec2f33 diff --git a/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 b/sql/hive/src/test/resources/golden/udf_equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 rename to sql/hive/src/test/resources/golden/udf_equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca diff --git a/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 b/sql/hive/src/test/resources/golden/udf_equal-6-878650cf21e9360a07d204c8ffb0cde7 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 rename to sql/hive/src/test/resources/golden/udf_equal-6-878650cf21e9360a07d204c8ffb0cde7 diff --git a/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e b/sql/hive/src/test/resources/golden/udf_equal-7-1635ef051fecdfc7891d9f5a9a3a545e similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e rename to sql/hive/src/test/resources/golden/udf_equal-7-1635ef051fecdfc7891d9f5a9a3a545e diff --git a/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 b/sql/hive/src/test/resources/golden/udf_equal-8-276101b04b10b7cd6d59061a8cbf42d2 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 rename to sql/hive/src/test/resources/golden/udf_equal-8-276101b04b10b7cd6d59061a8cbf42d2 diff --git a/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a b/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a index 9b3dbf316d1fb..2a5080be93ac4 100644 --- a/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a +++ b/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a @@ -1 +1 @@ -exp(x) - Returns e to the power of x \ No newline at end of file +exp(x) - Returns e to the power of x diff --git a/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 b/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 index a42c95bb8d213..b5a4d037f4014 100644 --- a/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 +++ b/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 @@ -1,4 +1,4 @@ exp(x) - Returns e to the power of x Example: > SELECT exp(0) FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 b/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 deleted file mode 100644 index a30bc26f5ba58..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 +++ /dev/null @@ -1 +0,0 @@ -field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udf_field-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 rename to sql/hive/src/test/resources/golden/udf_field-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 b/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 new file mode 100644 index 0000000000000..2e6133785ac7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 @@ -0,0 +1 @@ +field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found diff --git a/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f b/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f deleted file mode 100644 index bb55c0a1db1f5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f +++ /dev/null @@ -1,2 +0,0 @@ -field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found -All primitive types are supported, arguments are compared using str.equals(x). If str is NULL, the return value is 0. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 b/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 deleted file mode 100644 index 275f46482425d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 +++ /dev/null @@ -1,2 +0,0 @@ -86 val_86 0 0 2 1 1 0 0 -66 val_66 1 1 0 0 0 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-10-ca9db7e6bb687606bc273d1f6c191035 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_field-10-ca9db7e6bb687606bc273d1f6c191035 diff --git a/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 b/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 new file mode 100644 index 0000000000000..a13456f1bfdda --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 @@ -0,0 +1,2 @@ +86 val_86 0 0 2 1 1 0 0 +66 val_66 1 1 0 0 0 0 0 diff --git a/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f b/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f new file mode 100644 index 0000000000000..f9d418fee7b53 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f @@ -0,0 +1,2 @@ +field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found +All primitive types are supported, arguments are compared using str.equals(x). If str is NULL, the return value is 0. diff --git a/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b b/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b deleted file mode 100644 index c42823854fb0f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b +++ /dev/null @@ -1 +0,0 @@ -0 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 b/sql/hive/src/test/resources/golden/udf_field-3-5960d42257b272f6ba043406229bbf26 similarity index 100% rename from sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 rename to sql/hive/src/test/resources/golden/udf_field-3-5960d42257b272f6ba043406229bbf26 diff --git a/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 b/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 deleted file mode 100644 index 5869234249808..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 +++ /dev/null @@ -1 +0,0 @@ -1 2 3 4 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 b/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 new file mode 100644 index 0000000000000..e5449f0bfa473 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 @@ -0,0 +1 @@ +1 2 3 4 4 diff --git a/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 b/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 deleted file mode 100644 index 5869234249808..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 +++ /dev/null @@ -1 +0,0 @@ -1 2 3 4 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 b/sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b b/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b new file mode 100644 index 0000000000000..e5449f0bfa473 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b @@ -0,0 +1 @@ +1 2 3 4 4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-6-2d7c5cbe891c4a9dda34f9842f8e0828 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_field-6-2d7c5cbe891c4a9dda34f9842f8e0828 diff --git a/sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 b/sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-7-3fd8b0c333acdf28c676315b03e2e10 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_field-7-3fd8b0c333acdf28c676315b03e2e10 diff --git a/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef b/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef deleted file mode 100644 index a76563207da24..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef +++ /dev/null @@ -1,2 +0,0 @@ -86 val_86 0 0 2 0 0 0 0 0 2 0 -66 val_66 1 1 0 0 0 1 0 0 2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 b/sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef b/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef new file mode 100644 index 0000000000000..9af9d61b8b135 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef @@ -0,0 +1,2 @@ +86 val_86 0 0 2 0 0 0 0 0 2 0 +66 val_66 1 1 0 0 0 1 0 0 2 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 b/sql/hive/src/test/resources/golden/udf_field-9-66dc6c81db0ac9b2075783b0d8976083 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 rename to sql/hive/src/test/resources/golden/udf_field-9-66dc6c81db0ac9b2075783b0d8976083 diff --git a/sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 b/sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 b/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 deleted file mode 100644 index f14679978b79e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 +++ /dev/null @@ -1 +0,0 @@ -find_in_set(str,str_array) - Returns the first occurrence of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_find_in_set-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to sql/hive/src/test/resources/golden/udf_find_in_set-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 b/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 new file mode 100644 index 0000000000000..342fb7fad55b0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 @@ -0,0 +1 @@ +find_in_set(str,str_array) - Returns the first occurrence of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas. diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 b/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 deleted file mode 100644 index 1bebc68416bff..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 +++ /dev/null @@ -1,7 +0,0 @@ -find_in_set(str,str_array) - Returns the first occurrence of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas. -Example: - > SELECT find_in_set('ab','abc,b,ab,c,def') FROM src LIMIT 1; - 3 - > SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,956')=0; - 311 val_311 - 128 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c b/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 b/sql/hive/src/test/resources/golden/udf_find_in_set-10-df21f44247d7275a292520c1605c4aab similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 rename to sql/hive/src/test/resources/golden/udf_find_in_set-10-df21f44247d7275a292520c1605c4aab diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a b/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e b/sql/hive/src/test/resources/golden/udf_find_in_set-11-91f8c37820f31d0d1b16029a59a185ad similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e rename to sql/hive/src/test/resources/golden/udf_find_in_set-11-91f8c37820f31d0d1b16029a59a185ad diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 b/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 b/sql/hive/src/test/resources/golden/udf_find_in_set-12-692f41c998bbc6bec0f374eae4356739 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 rename to sql/hive/src/test/resources/golden/udf_find_in_set-12-692f41c998bbc6bec0f374eae4356739 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a b/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a deleted file mode 100644 index e440e5c842586..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a +++ /dev/null @@ -1 +0,0 @@ -3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 b/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 b/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 b/sql/hive/src/test/resources/golden/udf_find_in_set-14-8e410ecfad2d408ad7d2554ccd3a6621 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 rename to sql/hive/src/test/resources/golden/udf_find_in_set-14-8e410ecfad2d408ad7d2554ccd3a6621 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb b/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_find_in_set-15-c50e6ff95c05bfa854b33b03db858cd9 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/udf_find_in_set-15-c50e6ff95c05bfa854b33b03db858cd9 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/udf_find_in_set-16-8e17f41ae6e8b1075af4790a8fd88e13 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 rename to sql/hive/src/test/resources/golden/udf_find_in_set-16-8e17f41ae6e8b1075af4790a8fd88e13 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 b/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 b/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 deleted file mode 100644 index 99f516951ae7d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 +++ /dev/null @@ -1,2 +0,0 @@ -311 val_311 -128 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_find_in_set-17-fe61f992f2d971d006155bdec3143803 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to sql/hive/src/test/resources/golden/udf_find_in_set-17-fe61f992f2d971d006155bdec3143803 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 b/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 new file mode 100644 index 0000000000000..01228944b05a5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 @@ -0,0 +1,2 @@ +311 val_311 +128 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 b/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 new file mode 100644 index 0000000000000..d856144af1f86 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 @@ -0,0 +1,7 @@ +find_in_set(str,str_array) - Returns the first occurrence of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas. +Example: + > SELECT find_in_set('ab','abc,b,ab,c,def') FROM src LIMIT 1; + 3 + > SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,956')=0; + 311 val_311 + 128 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 b/sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e b/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e deleted file mode 100644 index 5817cbcff62ac..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e +++ /dev/null @@ -1,25 +0,0 @@ -1 -NULL -1 -NULL -NULL -NULL -1 -1 -1 -NULL -NULL -NULL -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -NULL -NULL -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_find_in_set-3-b3823bdc04a7f98951b55c3e30d2a772 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb rename to sql/hive/src/test/resources/golden/udf_find_in_set-3-b3823bdc04a7f98951b55c3e30d2a772 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e b/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e new file mode 100644 index 0000000000000..df725cb2c6aa7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e @@ -0,0 +1,25 @@ +1 +NULL +1 +NULL +NULL +NULL +1 +1 +1 +NULL +NULL +NULL +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +NULL +NULL +NULL diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b b/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/sql/hive/src/test/resources/golden/udf_find_in_set-5-6f25b5bba89e1fcae171f5d595acc4ee similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 rename to sql/hive/src/test/resources/golden/udf_find_in_set-5-6f25b5bba89e1fcae171f5d595acc4ee diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f b/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 b/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 deleted file mode 100644 index e440e5c842586..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 +++ /dev/null @@ -1 +0,0 @@ -3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba b/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 b/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 b/sql/hive/src/test/resources/golden/udf_find_in_set-7-730d5e95ef748ad946eceefbcd633826 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 rename to sql/hive/src/test/resources/golden/udf_find_in_set-7-730d5e95ef748ad946eceefbcd633826 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 b/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_find_in_set-8-ea11724531f191940e455d13878a0e69 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to sql/hive/src/test/resources/golden/udf_find_in_set-8-ea11724531f191940e455d13878a0e69 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 b/sql/hive/src/test/resources/golden/udf_find_in_set-9-81f9999ed1b063ce7f17d89bd0162777 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 rename to sql/hive/src/test/resources/golden/udf_find_in_set-9-81f9999ed1b063ce7f17d89bd0162777 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 b/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 b/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 index 9d15b5f5956d7..467e25bc261c2 100644 --- a/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 +++ b/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 @@ -1 +1 @@ -There is no documentation for function 'float' \ No newline at end of file +There is no documentation for function 'float' diff --git a/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b b/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b index 9d15b5f5956d7..467e25bc261c2 100644 --- a/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b +++ b/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b @@ -1 +1 @@ -There is no documentation for function 'float' \ No newline at end of file +There is no documentation for function 'float' diff --git a/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 b/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 index c76710bfcc1b5..de1563b40b836 100644 --- a/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 +++ b/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 @@ -1 +1 @@ -floor(x) - Find the largest integer not greater than x \ No newline at end of file +floor(x) - Find the largest integer not greater than x diff --git a/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 b/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 index 3f73eea16e183..ab6951202eb2a 100644 --- a/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 +++ b/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 @@ -3,4 +3,4 @@ Example: > SELECT floor(-0.1) FROM src LIMIT 1; -1 > SELECT floor(5) FROM src LIMIT 1; - 5 \ No newline at end of file + 5 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/udf_format_number-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to sql/hive/src/test/resources/golden/udf_format_number-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 b/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 deleted file mode 100644 index c981e1726c070..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 +++ /dev/null @@ -1 +0,0 @@ -format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_format_number-1-e86d559aeb84a4cc017a103182c22bfb similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb rename to sql/hive/src/test/resources/golden/udf_format_number-1-e86d559aeb84a4cc017a103182c22bfb diff --git a/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 b/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 new file mode 100644 index 0000000000000..8ee27a864b6dd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 @@ -0,0 +1 @@ +-9,223,372,036,854,775,807.0000000000 9,223,372,036,854,775,807.00000000000000000000 0.000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000005 179,769,313,486,231,570,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 b/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 new file mode 100644 index 0000000000000..14a40602519b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 @@ -0,0 +1 @@ +format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT diff --git a/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd b/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd deleted file mode 100644 index b2aa527ca0ddb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd +++ /dev/null @@ -1,4 +0,0 @@ -format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT -Example: - > SELECT format_number(12332.123456, 4) FROM src LIMIT 1; - '12,332.1235' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd b/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd new file mode 100644 index 0000000000000..def95a79e7375 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd @@ -0,0 +1,4 @@ +format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT +Example: + > SELECT format_number(12332.123456, 4) FROM src LIMIT 1; + '12,332.1235' diff --git a/sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c b/sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 b/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 deleted file mode 100644 index 89e118cc62bf3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 +++ /dev/null @@ -1 +0,0 @@ -12,332.1235 12,332.1000 12,332 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_format_number-4-7969ffc4e80f7214a8eead8e1084368a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb rename to sql/hive/src/test/resources/golden/udf_format_number-4-7969ffc4e80f7214a8eead8e1084368a diff --git a/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 b/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 deleted file mode 100644 index 33e21fa7dbfc4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 +++ /dev/null @@ -1 +0,0 @@ -0.123456789000 12,345,678.12346 1,234,567.1234568 123,456 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc b/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc new file mode 100644 index 0000000000000..055b84b4b35d8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc @@ -0,0 +1 @@ +12,332.1235 12,332.1000 12,332 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 b/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 new file mode 100644 index 0000000000000..9f12f9921318d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 @@ -0,0 +1 @@ +0.123456789000 12,345,678.12346 1,234,567.1234568 123,456 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 b/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 deleted file mode 100644 index 07b05cd16f5a3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 +++ /dev/null @@ -1 +0,0 @@ --123,456 -1,234,567.12 -0.123456789000000 -12,345.1235 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 b/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 new file mode 100644 index 0000000000000..032768d688943 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 @@ -0,0 +1 @@ +-123,456 -1,234,567.12 -0.123456789000000 -12,345.1235 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 b/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 deleted file mode 100644 index 2b600a6a83aa8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 +++ /dev/null @@ -1 +0,0 @@ -0.0000 0.0 0.0 0.0 -0.0000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea b/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea new file mode 100644 index 0000000000000..8077e5a60e4ef --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea @@ -0,0 +1 @@ +0.0000 0.0 0.0 0.0 -0.0000 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e b/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e deleted file mode 100644 index d027b2cb0b94a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e +++ /dev/null @@ -1 +0,0 @@ -0 1.0000 12.00 123.00000 1,234.0000000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 b/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 deleted file mode 100644 index afdec63c8dfca..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 +++ /dev/null @@ -1 +0,0 @@ --9,223,372,036,854,775,807.0000000000 9,223,372,036,854,775,807.00000000000000000000 0.000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000005 179,769,313,486,231,570,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 b/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 new file mode 100644 index 0000000000000..f9f98b94234f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 @@ -0,0 +1 @@ +0 1.0000 12.00 123.00000 1,234.0000000 diff --git a/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f b/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f index 10ce106f121ba..38550ea419625 100644 --- a/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f +++ b/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f @@ -1 +1 @@ -from_unixtime(unix_time, format) - returns unix_time in the specified format \ No newline at end of file +from_unixtime(unix_time, format) - returns unix_time in the specified format diff --git a/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 b/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 index 20f4d2b35c49a..ef15f822d80f5 100644 --- a/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 +++ b/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 @@ -1,4 +1,4 @@ from_unixtime(unix_time, format) - returns unix_time in the specified format Example: > SELECT from_unixtime(0, 'yyyy-MM-dd HH:mm:ss') FROM src LIMIT 1; - '1970-01-01 00:00:00' \ No newline at end of file + '1970-01-01 00:00:00' diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f b/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f deleted file mode 100644 index 4e4f3f7255fd3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f +++ /dev/null @@ -1 +0,0 @@ -get_json_object(json_txt, path) - Extract a json object from path diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 b/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 deleted file mode 100644 index 0e1dcf934a023..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 +++ /dev/null @@ -1,16 +0,0 @@ -get_json_object(json_txt, path) - Extract a json object from path -Extract json object from a json string based on json path specified, and return json string of the extracted json object. It will return null if the input json string is invalid. -A limited version of JSONPath supported: - $ : Root object - . : Child operator - [] : Subscript operator for array - * : Wildcard for [] -Syntax not supported that's worth noticing: - '' : Zero length string as key - .. : Recursive descent - &#064; : Current object/element - () : Script expression - ?() : Filter (script) expression. - [,] : Union operator - [start:end:step] : array slice operator - diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 b/sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/udf_greaterthan-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 rename to sql/hive/src/test/resources/golden/udf_greaterthan-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 b/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 deleted file mode 100644 index 54d6096d07bbe..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 +++ /dev/null @@ -1 +0,0 @@ -a > b - Returns TRUE if a is greater than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c b/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c deleted file mode 100644 index 54d6096d07bbe..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c +++ /dev/null @@ -1 +0,0 @@ -a > b - Returns TRUE if a is greater than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 b/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 new file mode 100644 index 0000000000000..1eec522da2a1e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 @@ -0,0 +1 @@ +a > b - Returns TRUE if a is greater than b diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f b/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f deleted file mode 100644 index 679b0376125f0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f +++ /dev/null @@ -1 +0,0 @@ -true false false false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c b/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c new file mode 100644 index 0000000000000..1eec522da2a1e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c @@ -0,0 +1 @@ +a > b - Returns TRUE if a is greater than b diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 b/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 new file mode 100644 index 0000000000000..096c64e2afd93 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 @@ -0,0 +1 @@ +true false false false diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 rename to sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a deleted file mode 100644 index abf7dfdab730c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a +++ /dev/null @@ -1 +0,0 @@ -a >= b - Returns TRUE if a is not smaller than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 deleted file mode 100644 index abf7dfdab730c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 +++ /dev/null @@ -1 +0,0 @@ -a >= b - Returns TRUE if a is not smaller than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a new file mode 100644 index 0000000000000..d5422146acd0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a @@ -0,0 +1 @@ +a >= b - Returns TRUE if a is not smaller than b diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 new file mode 100644 index 0000000000000..d5422146acd0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 @@ -0,0 +1 @@ +a >= b - Returns TRUE if a is not smaller than b diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 deleted file mode 100644 index 1fb1894fc21f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 +++ /dev/null @@ -1 +0,0 @@ -true false true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 new file mode 100644 index 0000000000000..435a5f7b0efb3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 @@ -0,0 +1 @@ +true false true true diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/udf_hash-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 rename to sql/hive/src/test/resources/golden/udf_hash-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 b/sql/hive/src/test/resources/golden/udf_hash-1-b9e3a3986320d275982797140edfccf4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 rename to sql/hive/src/test/resources/golden/udf_hash-1-b9e3a3986320d275982797140edfccf4 diff --git a/sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a b/sql/hive/src/test/resources/golden/udf_hash-2-a18646b51501d0b1beb967dc79afbd1a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a rename to sql/hive/src/test/resources/golden/udf_hash-2-a18646b51501d0b1beb967dc79afbd1a diff --git a/sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 b/sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/udf_hash-3-2646a87ce26c383a9dafea9b56281ab7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 rename to sql/hive/src/test/resources/golden/udf_hash-3-2646a87ce26c383a9dafea9b56281ab7 diff --git a/sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae b/sql/hive/src/test/resources/golden/udf_hash-4-d1368c2e3cd113e46202156b44811987 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae rename to sql/hive/src/test/resources/golden/udf_hash-4-d1368c2e3cd113e46202156b44811987 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/udf_hex-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 rename to sql/hive/src/test/resources/golden/udf_hex-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 b/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 deleted file mode 100644 index f87a6117eacf7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 +++ /dev/null @@ -1 +0,0 @@ -hex(n, bin, or str) - Convert the argument to hexadecimal \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 b/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 new file mode 100644 index 0000000000000..c45cea8bc13b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 @@ -0,0 +1 @@ +hex(n, bin, or str) - Convert the argument to hexadecimal diff --git a/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c b/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c deleted file mode 100644 index e8cee4afc3a80..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c +++ /dev/null @@ -1,8 +0,0 @@ -hex(n, bin, or str) - Convert the argument to hexadecimal -If the argument is a string, returns two hex digits for each character in the string. -If the argument is a number or binary, returns the hexadecimal representation. -Example: - > SELECT hex(17) FROM src LIMIT 1; - 'H1' - > SELECT hex('Facebook') FROM src LIMIT 1; - '46616365626F6F6B' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 b/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 deleted file mode 100644 index 34eb75a6c784a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 +++ /dev/null @@ -1 +0,0 @@ -46616365626F6F6B 00 71776572747975696F706173646667686A6B6C \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c b/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c new file mode 100644 index 0000000000000..bcc46336d0fd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c @@ -0,0 +1,8 @@ +hex(n, bin, or str) - Convert the argument to hexadecimal +If the argument is a string, returns two hex digits for each character in the string. +If the argument is a number or binary, returns the hexadecimal representation. +Example: + > SELECT hex(17) FROM src LIMIT 1; + 'H1' + > SELECT hex('Facebook') FROM src LIMIT 1; + '46616365626F6F6B' diff --git a/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 b/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 new file mode 100644 index 0000000000000..b0ffe57c8e161 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 @@ -0,0 +1 @@ +46616365626F6F6B 00 71776572747975696F706173646667686A6B6C diff --git a/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf b/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf deleted file mode 100644 index 2a45250c67a63..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf +++ /dev/null @@ -1 +0,0 @@ -1 0 FACEB005 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 b/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 new file mode 100644 index 0000000000000..8c56faa0f9db9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 @@ -0,0 +1 @@ +1 0 FACEB005 diff --git a/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea b/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea deleted file mode 100644 index 50d9557967ac4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea +++ /dev/null @@ -1 +0,0 @@ -FFFFFFFFFFFFFFFB \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e b/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e new file mode 100644 index 0000000000000..b766160c67704 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFB diff --git a/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce b/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce deleted file mode 100644 index 48911456dc339..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce +++ /dev/null @@ -1 +0,0 @@ -hour(date) - Returns the hour of date diff --git a/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 b/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 deleted file mode 100644 index be1a966635202..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 +++ /dev/null @@ -1,7 +0,0 @@ -hour(date) - Returns the hour of date -date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'. -Example: - > SELECT hour('2009-07-30 12:58:59') FROM src LIMIT 1; - 12 - > SELECT hour('12:58:59') FROM src LIMIT 1; - 12 diff --git a/sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a b/sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 b/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 deleted file mode 100644 index 1a35f9f158133..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 +++ /dev/null @@ -1 +0,0 @@ -13 13 NULL diff --git a/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a b/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a deleted file mode 100644 index b1bd38b62a080..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a +++ /dev/null @@ -1 +0,0 @@ -13 diff --git a/sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/udf_if-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a rename to sql/hive/src/test/resources/golden/udf_if-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c b/sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c similarity index 100% rename from sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c rename to sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c diff --git a/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a b/sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a rename to sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a diff --git a/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 b/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 b/sql/hive/src/test/resources/golden/udf_if-3-59e90bb74481aaf35480076806daf365 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 rename to sql/hive/src/test/resources/golden/udf_if-3-59e90bb74481aaf35480076806daf365 diff --git a/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca b/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f b/sql/hive/src/test/resources/golden/udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb similarity index 100% rename from sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f rename to sql/hive/src/test/resources/golden/udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c b/sql/hive/src/test/resources/golden/udf_if-5-841a8068d35a42179d3654e1a2848c43 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c rename to sql/hive/src/test/resources/golden/udf_if-5-841a8068d35a42179d3654e1a2848c43 diff --git a/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc b/sql/hive/src/test/resources/golden/udf_if-6-508f9140dd33931c7b9ad336dceb32cf similarity index 100% rename from sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc rename to sql/hive/src/test/resources/golden/udf_if-6-508f9140dd33931c7b9ad336dceb32cf diff --git a/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 b/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 deleted file mode 100644 index ee6e42ce6a83d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 +++ /dev/null @@ -1 +0,0 @@ -true false true true false true true true NULL NULL true diff --git a/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 b/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 deleted file mode 100644 index 993d93304f95e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 +++ /dev/null @@ -1,3 +0,0 @@ -238 -86 -238 diff --git a/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 b/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 deleted file mode 100644 index cd15a08a539ab..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 +++ /dev/null @@ -1 +0,0 @@ -in_file(str, filename) - Returns true if str appears in the file diff --git a/sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 b/sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 b/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 deleted file mode 100644 index df5a0561fb8f5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 +++ /dev/null @@ -1 +0,0 @@ -Function '`index`' does not exist. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_index-0-e91e3e5a22029b9b979ccbbef97add66 similarity index 100% rename from sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_index-0-e91e3e5a22029b9b979ccbbef97add66 diff --git a/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 b/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 deleted file mode 100644 index df5a0561fb8f5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 +++ /dev/null @@ -1 +0,0 @@ -Function '`index`' does not exist. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 b/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 new file mode 100644 index 0000000000000..bb0a912824bbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 @@ -0,0 +1 @@ +Function '`index`' does not exist. diff --git a/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 b/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 new file mode 100644 index 0000000000000..bb0a912824bbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 @@ -0,0 +1 @@ +Function '`index`' does not exist. diff --git a/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 b/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 deleted file mode 100644 index ae27b5efea045..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 +++ /dev/null @@ -1 +0,0 @@ -instr(str, substr) - Returns the index of the first occurance of substr in str \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/udf_instr-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 rename to sql/hive/src/test/resources/golden/udf_instr-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 b/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 new file mode 100644 index 0000000000000..06461b525b058 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 @@ -0,0 +1 @@ +instr(str, substr) - Returns the index of the first occurance of substr in str diff --git a/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 b/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 deleted file mode 100644 index 35de2f0d8653f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 +++ /dev/null @@ -1,4 +0,0 @@ -instr(str, substr) - Returns the index of the first occurance of substr in str -Example: - > SELECT instr('Facebook', 'boo') FROM src LIMIT 1; - 5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 b/sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 b/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 new file mode 100644 index 0000000000000..5a8c34271f443 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 @@ -0,0 +1,4 @@ +instr(str, substr) - Returns the index of the first occurance of substr in str +Example: + > SELECT instr('Facebook', 'boo') FROM src LIMIT 1; + 5 diff --git a/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 b/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 deleted file mode 100644 index 4d34e6df7039b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 +++ /dev/null @@ -1 +0,0 @@ -1 0 2 2 0 0 2 3 4 2 3 NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/udf_instr-3-c40fbd09410b11388ce7a6e9bea5846f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 rename to sql/hive/src/test/resources/golden/udf_instr-3-c40fbd09410b11388ce7a6e9bea5846f diff --git a/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 b/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 new file mode 100644 index 0000000000000..8883b1a631ab7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 @@ -0,0 +1 @@ +1 0 2 2 0 0 2 3 4 2 3 NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba b/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba index cda33a8e2e555..5549ad557f6b2 100644 --- a/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba +++ b/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba @@ -1 +1 @@ -There is no documentation for function 'int' \ No newline at end of file +There is no documentation for function 'int' diff --git a/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 b/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 index cda33a8e2e555..5549ad557f6b2 100644 --- a/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 +++ b/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 @@ -1 +1 @@ -There is no documentation for function 'int' \ No newline at end of file +There is no documentation for function 'int' diff --git a/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 b/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 index 723b5aaf2af6b..4ae9ea2a16e87 100644 --- a/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 +++ b/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 @@ -1 +1 @@ -isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file +isnotnull a - Returns true if a is not NULL and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 b/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 index 723b5aaf2af6b..4ae9ea2a16e87 100644 --- a/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 +++ b/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 @@ -1 +1 @@ -isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file +isnotnull a - Returns true if a is not NULL and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 b/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 index 99510c6f1db15..237cdf99168d0 100644 --- a/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 +++ b/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 @@ -1 +1 @@ -isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file +isnull a - Returns true if a is NULL and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 b/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 index 99510c6f1db15..237cdf99168d0 100644 --- a/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 +++ b/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 @@ -1 +1 @@ -isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file +isnull a - Returns true if a is NULL and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 deleted file mode 100644 index 99510c6f1db15..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 +++ /dev/null @@ -1 +0,0 @@ -isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 deleted file mode 100644 index 99510c6f1db15..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 +++ /dev/null @@ -1 +0,0 @@ -isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 deleted file mode 100644 index 723b5aaf2af6b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 +++ /dev/null @@ -1 +0,0 @@ -isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 deleted file mode 100644 index 723b5aaf2af6b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 +++ /dev/null @@ -1 +0,0 @@ -isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 deleted file mode 100644 index eedfbc67d50c7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 +++ /dev/null @@ -1 +0,0 @@ -true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b deleted file mode 100644 index eedfbc67d50c7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b +++ /dev/null @@ -1 +0,0 @@ -true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 b/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 deleted file mode 100644 index b703a30abffba..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 +++ /dev/null @@ -1 +0,0 @@ -java_method(class,method[,arg1[,arg2..]]) calls method with reflection \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a b/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a deleted file mode 100644 index 07375dc92d625..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a +++ /dev/null @@ -1,3 +0,0 @@ -java_method(class,method[,arg1[,arg2..]]) calls method with reflection -Synonyms: reflect -Use this UDF to call Java methods by matching the argument signature diff --git a/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 b/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd b/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd deleted file mode 100644 index 9b93703dae806..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd +++ /dev/null @@ -1 +0,0 @@ -1 true 3 2 3 2.718281828459045 1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad b/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad index ec6779df2818d..afe1bf6cd351a 100644 --- a/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad +++ b/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad @@ -1 +1 @@ -lcase(str) - Returns str with all characters changed to lowercase \ No newline at end of file +lcase(str) - Returns str with all characters changed to lowercase diff --git a/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e b/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e index 4da30f6c23a1b..191001deca9f7 100644 --- a/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e +++ b/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e @@ -2,4 +2,4 @@ lcase(str) - Returns str with all characters changed to lowercase Synonyms: lower Example: > SELECT lcase('Facebook') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b b/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b deleted file mode 100644 index d4d2bd508b44f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b +++ /dev/null @@ -1 +0,0 @@ -length(str | binary) - Returns the length of str or number of bytes in binary data \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_length-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_length-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b b/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b new file mode 100644 index 0000000000000..3e2bae7430e67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b @@ -0,0 +1 @@ +length(str | binary) - Returns the length of str or number of bytes in binary data diff --git a/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 b/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 deleted file mode 100644 index 79c1c54639266..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 +++ /dev/null @@ -1,4 +0,0 @@ -length(str | binary) - Returns the length of str or number of bytes in binary data -Example: - > SELECT length('Facebook') FROM src LIMIT 1; - 8 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea b/sql/hive/src/test/resources/golden/udf_length-10-de456a5765db4a06110d9483985aa4a6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea rename to sql/hive/src/test/resources/golden/udf_length-10-de456a5765db4a06110d9483985aa4a6 diff --git a/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 b/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 b/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 b/sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 b/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 new file mode 100644 index 0000000000000..f1d3b1648718d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 @@ -0,0 +1,4 @@ +length(str | binary) - Returns the length of str or number of bytes in binary data +Example: + > SELECT length('Facebook') FROM src LIMIT 1; + 8 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 b/sql/hive/src/test/resources/golden/udf_length-3-af46cb6887618240836eaf5be8afbba6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 rename to sql/hive/src/test/resources/golden/udf_length-3-af46cb6887618240836eaf5be8afbba6 diff --git a/sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d b/sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 b/sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 b/sql/hive/src/test/resources/golden/udf_length-4-dcd6404afce1103d5054527e6c216d6d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 rename to sql/hive/src/test/resources/golden/udf_length-4-dcd6404afce1103d5054527e6c216d6d diff --git a/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 6ecb2baabb297..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,25 +0,0 @@ -7 -0 -7 -6 -7 -7 -7 -7 -6 -7 -7 -7 -7 -7 -7 -0 -0 -6 -0 -7 -7 -7 -0 -0 -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 b/sql/hive/src/test/resources/golden/udf_length-5-ba49ba4e6505c74bc33b5003f3930c43 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 rename to sql/hive/src/test/resources/golden/udf_length-5-ba49ba4e6505c74bc33b5003f3930c43 diff --git a/sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..c77515b2f479b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,25 @@ +7 +0 +7 +6 +7 +7 +7 +7 +6 +7 +7 +7 +7 +7 +7 +0 +0 +6 +0 +7 +7 +7 +0 +0 +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 b/sql/hive/src/test/resources/golden/udf_length-7-460dffb0f1ab0ac0ebc4fd545809aa9a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 rename to sql/hive/src/test/resources/golden/udf_length-7-460dffb0f1ab0ac0ebc4fd545809aa9a diff --git a/sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f b/sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf b/sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 b/sql/hive/src/test/resources/golden/udf_length-8-8f28e6c488df47d89dca670f04a7563f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 rename to sql/hive/src/test/resources/golden/udf_length-8-8f28e6c488df47d89dca670f04a7563f diff --git a/sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 b/sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 b/sql/hive/src/test/resources/golden/udf_length-9-e41b220da98996f997b26ba7ef457a84 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 rename to sql/hive/src/test/resources/golden/udf_length-9-e41b220da98996f997b26ba7ef457a84 diff --git a/sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/udf_lessthan-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 rename to sql/hive/src/test/resources/golden/udf_lessthan-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 b/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 deleted file mode 100644 index b43707d550c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 +++ /dev/null @@ -1 +0,0 @@ -a < b - Returns TRUE if a is less than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 b/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 deleted file mode 100644 index b43707d550c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 +++ /dev/null @@ -1 +0,0 @@ -a < b - Returns TRUE if a is less than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 b/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 new file mode 100644 index 0000000000000..d7621677e2fce --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 @@ -0,0 +1 @@ +a < b - Returns TRUE if a is less than b diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 b/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 deleted file mode 100644 index 4364012093724..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 +++ /dev/null @@ -1 +0,0 @@ -false true false false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 b/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 new file mode 100644 index 0000000000000..d7621677e2fce --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 @@ -0,0 +1 @@ +a < b - Returns TRUE if a is less than b diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 b/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 new file mode 100644 index 0000000000000..5926a6ac7746d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 @@ -0,0 +1 @@ +false true false false diff --git a/sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 rename to sql/hive/src/test/resources/golden/udf_lessthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b b/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b deleted file mode 100644 index c36acd7ce80bc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b +++ /dev/null @@ -1 +0,0 @@ -a <= b - Returns TRUE if a is not greater than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 deleted file mode 100644 index c36acd7ce80bc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 +++ /dev/null @@ -1 +0,0 @@ -a <= b - Returns TRUE if a is not greater than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b new file mode 100644 index 0000000000000..7f18733b6e47e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b @@ -0,0 +1 @@ +a <= b - Returns TRUE if a is not greater than b diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c deleted file mode 100644 index 38e013b6914c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c +++ /dev/null @@ -1 +0,0 @@ -false true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 new file mode 100644 index 0000000000000..7f18733b6e47e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 @@ -0,0 +1 @@ +a <= b - Returns TRUE if a is not greater than b diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 new file mode 100644 index 0000000000000..fc2c7b1da1920 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 @@ -0,0 +1 @@ +false true true true diff --git a/sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/udf_like-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 rename to sql/hive/src/test/resources/golden/udf_like-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 b/sql/hive/src/test/resources/golden/udf_like-1-e0ba9a953e50554bdcbc55585cffde09 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 rename to sql/hive/src/test/resources/golden/udf_like-1-e0ba9a953e50554bdcbc55585cffde09 diff --git a/sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a b/sql/hive/src/test/resources/golden/udf_like-2-9781f89d352c506e972ad2a1d58ec03a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a rename to sql/hive/src/test/resources/golden/udf_like-2-9781f89d352c506e972ad2a1d58ec03a diff --git a/sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 b/sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 b/sql/hive/src/test/resources/golden/udf_like-3-dbc46cb33f0dd356af03006d9492f8b7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 rename to sql/hive/src/test/resources/golden/udf_like-3-dbc46cb33f0dd356af03006d9492f8b7 diff --git a/sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc b/sql/hive/src/test/resources/golden/udf_like-4-bef03784eab9d5e8404fd24960dea4fc similarity index 100% rename from sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc rename to sql/hive/src/test/resources/golden/udf_like-4-bef03784eab9d5e8404fd24960dea4fc diff --git a/sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 b/sql/hive/src/test/resources/golden/udf_like-5-47bfd4d65090dab890b467ae06cf3bd5 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 rename to sql/hive/src/test/resources/golden/udf_like-5-47bfd4d65090dab890b467ae06cf3bd5 diff --git a/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 b/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 index 0b7372cb446d9..134096b19d8c3 100644 --- a/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 +++ b/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 @@ -1 +1 @@ -ln(x) - Returns the natural logarithm of x \ No newline at end of file +ln(x) - Returns the natural logarithm of x diff --git a/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 b/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 index bdf2f49ab17ca..c4a1a46f93370 100644 --- a/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 +++ b/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 @@ -1,4 +1,4 @@ ln(x) - Returns the natural logarithm of x Example: > SELECT ln(1) FROM src LIMIT 1; - 0 \ No newline at end of file + 0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_locate-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_locate-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e b/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e deleted file mode 100644 index 63b152162407b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e +++ /dev/null @@ -1 +0,0 @@ -locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e b/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e new file mode 100644 index 0000000000000..84bea329540d1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e @@ -0,0 +1 @@ +locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos diff --git a/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 b/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 deleted file mode 100644 index ea5465b67683e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 +++ /dev/null @@ -1,4 +0,0 @@ -locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos -Example: - > SELECT locate('bar', 'foobarbar', 5) FROM src LIMIT 1; - 7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c b/sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 b/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 new file mode 100644 index 0000000000000..092e12586b9e8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 @@ -0,0 +1,4 @@ +locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos +Example: + > SELECT locate('bar', 'foobarbar', 5) FROM src LIMIT 1; + 7 diff --git a/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 b/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 deleted file mode 100644 index cd97bbb17fa81..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 +++ /dev/null @@ -1 +0,0 @@ -1 0 2 2 4 4 0 0 2 3 4 2 3 NULL NULL 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 b/sql/hive/src/test/resources/golden/udf_locate-3-ce4a131f99dc9befa926027380b38dbb similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 rename to sql/hive/src/test/resources/golden/udf_locate-3-ce4a131f99dc9befa926027380b38dbb diff --git a/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b b/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b new file mode 100644 index 0000000000000..f98ccc3556b42 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b @@ -0,0 +1 @@ +1 0 2 2 4 4 0 0 2 3 4 2 3 NULL NULL 0 0 diff --git a/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a b/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a index e0dba6b06d9a4..c2c4ca2cd4385 100644 --- a/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a +++ b/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a @@ -1 +1 @@ -log([b], x) - Returns the logarithm of x with base b \ No newline at end of file +log([b], x) - Returns the logarithm of x with base b diff --git a/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 b/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 index 3dedfb4ddfd88..d6bc6ebbfd2d1 100644 --- a/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 +++ b/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 @@ -1,4 +1,4 @@ log([b], x) - Returns the logarithm of x with base b Example: > SELECT log(13, 13) FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f b/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f index 61749c48f158e..b9540eb85a360 100644 --- a/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f +++ b/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f @@ -1 +1 @@ -log10(x) - Returns the logarithm of x with base 10 \ No newline at end of file +log10(x) - Returns the logarithm of x with base 10 diff --git a/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 b/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 index 68ce39dd2b084..b71baf542f924 100644 --- a/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 +++ b/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 @@ -1,4 +1,4 @@ log10(x) - Returns the logarithm of x with base 10 Example: > SELECT log10(10) FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 b/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 index 177008a543627..8214317d87b0b 100644 --- a/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 +++ b/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 @@ -1 +1 @@ -log2(x) - Returns the logarithm of x with base 2 \ No newline at end of file +log2(x) - Returns the logarithm of x with base 2 diff --git a/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 b/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 index 4b856f91a7a14..579ad959cd884 100644 --- a/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 +++ b/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 @@ -1,4 +1,4 @@ log2(x) - Returns the logarithm of x with base 2 Example: > SELECT log2(2) FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 b/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 deleted file mode 100644 index d9d0dc035c9ab..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_udf_get_java_boolean 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaBoolean') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 b/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 index 9bfc44922ea38..de8e1518c93d7 100644 --- a/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 +++ b/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 @@ -1 +1 @@ -lower(str) - Returns str with all characters changed to lowercase \ No newline at end of file +lower(str) - Returns str with all characters changed to lowercase diff --git a/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c b/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c index da912776ab009..532f348d7da6f 100644 --- a/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c +++ b/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c @@ -2,4 +2,4 @@ lower(str) - Returns str with all characters changed to lowercase Synonyms: lcase Example: > SELECT lower('Facebook') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 b/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 index 9640d1794dcbc..8fbb5d4429d96 100644 --- a/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 +++ b/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 @@ -1 +1 @@ -abc 123 ABC 123 \ No newline at end of file +abc 123 ABC 123 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_lpad-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_lpad-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef b/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef deleted file mode 100644 index 672d2d07f68b5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef +++ /dev/null @@ -1 +0,0 @@ -lpad(str, len, pad) - Returns str, left-padded with pad to a length of len \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef b/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef new file mode 100644 index 0000000000000..e756eab5119d2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef @@ -0,0 +1 @@ +lpad(str, len, pad) - Returns str, left-padded with pad to a length of len diff --git a/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 b/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 deleted file mode 100644 index 66db6624a376a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 +++ /dev/null @@ -1,6 +0,0 @@ -lpad(str, len, pad) - Returns str, left-padded with pad to a length of len -If str is longer than len, the return value is shortened to len characters. -Example: - > SELECT lpad('hi', 5, '??') FROM src LIMIT 1; - '???hi' > SELECT lpad('hi', 1, '??') FROM src LIMIT 1; - 'h' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 b/sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 b/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 new file mode 100644 index 0000000000000..122971dbc2df9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 @@ -0,0 +1,6 @@ +lpad(str, len, pad) - Returns str, left-padded with pad to a length of len +If str is longer than len, the return value is shortened to len characters. +Example: + > SELECT lpad('hi', 5, '??') FROM src LIMIT 1; + '???hi' > SELECT lpad('hi', 1, '??') FROM src LIMIT 1; + 'h' diff --git a/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae b/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae deleted file mode 100644 index 63a7235050df3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae +++ /dev/null @@ -1 +0,0 @@ -h ...hi 1231hi \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f b/sql/hive/src/test/resources/golden/udf_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f rename to sql/hive/src/test/resources/golden/udf_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7 diff --git a/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 b/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 new file mode 100644 index 0000000000000..f75d0d891636a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 @@ -0,0 +1 @@ +h ...hi 1231hi diff --git a/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f b/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f index 0c9f6fd1c269e..419d80fe502a3 100644 --- a/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f +++ b/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f @@ -1 +1 @@ -ltrim(str) - Removes the leading space characters from str \ No newline at end of file +ltrim(str) - Removes the leading space characters from str diff --git a/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 b/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 index 3ee2e690d28c1..fc994e90aaa05 100644 --- a/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 +++ b/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 @@ -1,4 +1,4 @@ ltrim(str) - Removes the leading space characters from str Example: > SELECT ltrim(' facebook') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_map-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e rename to sql/hive/src/test/resources/golden/udf_map-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e b/sql/hive/src/test/resources/golden/udf_map-1-9feb9f29115f94b3bb4f6a36160bd17e similarity index 100% rename from sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e rename to sql/hive/src/test/resources/golden/udf_map-1-9feb9f29115f94b3bb4f6a36160bd17e diff --git a/sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf b/sql/hive/src/test/resources/golden/udf_map-2-1f8cd98df9bf7b2528506551fef87dcf similarity index 100% rename from sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf rename to sql/hive/src/test/resources/golden/udf_map-2-1f8cd98df9bf7b2528506551fef87dcf diff --git a/sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 b/sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 b/sql/hive/src/test/resources/golden/udf_map-3-be7b52baa973b8b59b7ca63fea19aa99 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 rename to sql/hive/src/test/resources/golden/udf_map-3-be7b52baa973b8b59b7ca63fea19aa99 diff --git a/sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 b/sql/hive/src/test/resources/golden/udf_map-4-60cb9c30285f7a9f99377ccbd143eb06 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 rename to sql/hive/src/test/resources/golden/udf_map-4-60cb9c30285f7a9f99377ccbd143eb06 diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa b/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa deleted file mode 100644 index b88b74b978b8f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa +++ /dev/null @@ -1 +0,0 @@ -map_keys(map) - Returns an unordered array containing the keys of the input map. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f b/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f deleted file mode 100644 index b88b74b978b8f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f +++ /dev/null @@ -1 +0,0 @@ -map_keys(map) - Returns an unordered array containing the keys of the input map. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e b/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e deleted file mode 100644 index 3a26a2e5e94d5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e +++ /dev/null @@ -1 +0,0 @@ -[1,2,3] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a b/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a deleted file mode 100644 index 9b4c194f58a8e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a +++ /dev/null @@ -1 +0,0 @@ -["b","a","c"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 b/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 deleted file mode 100644 index e811d1976a24c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 +++ /dev/null @@ -1 +0,0 @@ -map_values(map) - Returns an unordered array containing the values of the input map. diff --git a/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 b/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 deleted file mode 100644 index e811d1976a24c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 +++ /dev/null @@ -1 +0,0 @@ -map_values(map) - Returns an unordered array containing the values of the input map. diff --git a/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 b/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 deleted file mode 100644 index 9d804e48987c6..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 +++ /dev/null @@ -1 +0,0 @@ -["a","b","c"] diff --git a/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e b/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e deleted file mode 100644 index be56bd1673eaa..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e +++ /dev/null @@ -1 +0,0 @@ -[2,1,3] diff --git a/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f b/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f deleted file mode 100644 index c6cd9573ed8ed..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f +++ /dev/null @@ -1 +0,0 @@ -max(expr) - Returns the maximum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 b/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 deleted file mode 100644 index c6cd9573ed8ed..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 +++ /dev/null @@ -1 +0,0 @@ -max(expr) - Returns the maximum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 deleted file mode 100644 index 21096f0f69749..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 +++ /dev/null @@ -1 +0,0 @@ -{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 deleted file mode 100644 index 21096f0f69749..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 +++ /dev/null @@ -1 +0,0 @@ -{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 deleted file mode 100644 index 21096f0f69749..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 +++ /dev/null @@ -1 +0,0 @@ -{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 deleted file mode 100644 index 21096f0f69749..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 +++ /dev/null @@ -1 +0,0 @@ -{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 b/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 deleted file mode 100644 index 0d59cabcde8fb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 +++ /dev/null @@ -1 +0,0 @@ -min(expr) - Returns the minimum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 b/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 deleted file mode 100644 index 0d59cabcde8fb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 +++ /dev/null @@ -1 +0,0 @@ -min(expr) - Returns the minimum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 deleted file mode 100644 index b26a7e93c5d9f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 +++ /dev/null @@ -1 +0,0 @@ -{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 deleted file mode 100644 index b26a7e93c5d9f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 +++ /dev/null @@ -1 +0,0 @@ -{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 deleted file mode 100644 index b26a7e93c5d9f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 +++ /dev/null @@ -1 +0,0 @@ -{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 deleted file mode 100644 index b26a7e93c5d9f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 +++ /dev/null @@ -1 +0,0 @@ -{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee b/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee index b6bd6049a90b9..231e4f382566d 100644 --- a/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee +++ b/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee @@ -1 +1 @@ -minute(date) - Returns the minute of date \ No newline at end of file +minute(date) - Returns the minute of date diff --git a/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 b/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 index 64c3cda697e1e..ea842ea174ae4 100644 --- a/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 +++ b/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 @@ -4,4 +4,4 @@ Example: > SELECT minute('2009-07-30 12:58:59') FROM src LIMIT 1; 58 > SELECT minute('12:58:59') FROM src LIMIT 1; - 58 \ No newline at end of file + 58 diff --git a/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c b/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c index 5bd5433375d5c..e9ca4578409c8 100644 --- a/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c +++ b/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c @@ -1 +1 @@ -14 14 NULL \ No newline at end of file +14 14 NULL diff --git a/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 b/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 index 1bcf0c71008b0..6df0bfb4fecf8 100644 --- a/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 +++ b/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 @@ -1 +1 @@ -a % b - Returns the remainder when dividing a by b \ No newline at end of file +a % b - Returns the remainder when dividing a by b diff --git a/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 b/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 index 1bcf0c71008b0..6df0bfb4fecf8 100644 --- a/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 +++ b/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 @@ -1 +1 @@ -a % b - Returns the remainder when dividing a by b \ No newline at end of file +a % b - Returns the remainder when dividing a by b diff --git a/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee b/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee index b6bd6049a90b9..231e4f382566d 100644 --- a/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee +++ b/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee @@ -1 +1 @@ -minute(date) - Returns the minute of date \ No newline at end of file +minute(date) - Returns the minute of date diff --git a/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 b/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 index 64c3cda697e1e..ea842ea174ae4 100644 --- a/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 +++ b/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 @@ -4,4 +4,4 @@ Example: > SELECT minute('2009-07-30 12:58:59') FROM src LIMIT 1; 58 > SELECT minute('12:58:59') FROM src LIMIT 1; - 58 \ No newline at end of file + 58 diff --git a/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 b/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 deleted file mode 100644 index 3142f7f94be9e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 +++ /dev/null @@ -1 +0,0 @@ -negative a - Returns -a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_negative-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e rename to sql/hive/src/test/resources/golden/udf_negative-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 b/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 new file mode 100644 index 0000000000000..7ba1b77782f07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 @@ -0,0 +1 @@ +negative a - Returns -a diff --git a/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf b/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf deleted file mode 100644 index 3142f7f94be9e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf +++ /dev/null @@ -1 +0,0 @@ -negative a - Returns -a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 b/sql/hive/src/test/resources/golden/udf_negative-10-1cd28efecc0d26f463221195f5e39956 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 rename to sql/hive/src/test/resources/golden/udf_negative-10-1cd28efecc0d26f463221195f5e39956 diff --git a/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf b/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf new file mode 100644 index 0000000000000..7ba1b77782f07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf @@ -0,0 +1 @@ +negative a - Returns -a diff --git a/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c b/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c deleted file mode 100644 index 0f4a4dce9dd3d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c +++ /dev/null @@ -1 +0,0 @@ -a - b - Returns the difference a-b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c b/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c new file mode 100644 index 0000000000000..1ad8be1242f00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c @@ -0,0 +1 @@ +a - b - Returns the difference a-b diff --git a/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 deleted file mode 100644 index 0f4a4dce9dd3d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 +++ /dev/null @@ -1 +0,0 @@ -a - b - Returns the difference a-b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 new file mode 100644 index 0000000000000..1ad8be1242f00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 @@ -0,0 +1 @@ +a - b - Returns the difference a-b diff --git a/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c b/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a b/sql/hive/src/test/resources/golden/udf_negative-5-771e76b0acd8ddb128781da7819d0e47 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a rename to sql/hive/src/test/resources/golden/udf_negative-5-771e76b0acd8ddb128781da7819d0e47 diff --git a/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e b/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 b/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d b/sql/hive/src/test/resources/golden/udf_negative-6-f62c4a097c592871d896a7dc47c42f61 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d rename to sql/hive/src/test/resources/golden/udf_negative-6-f62c4a097c592871d896a7dc47c42f61 diff --git a/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b b/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 b/sql/hive/src/test/resources/golden/udf_negative-7-f838053f5ca5c8746dc299473dff0490 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 rename to sql/hive/src/test/resources/golden/udf_negative-7-f838053f5ca5c8746dc299473dff0490 diff --git a/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 b/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da b/sql/hive/src/test/resources/golden/udf_negative-8-f4f23aa6f634913d194a69261af8f3f6 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da rename to sql/hive/src/test/resources/golden/udf_negative-8-f4f23aa6f634913d194a69261af8f3f6 diff --git a/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa b/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 b/sql/hive/src/test/resources/golden/udf_negative-9-f6a78fa3ea0f519d0e4abc5be7a960e5 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 rename to sql/hive/src/test/resources/golden/udf_negative-9-f6a78fa3ea0f519d0e4abc5be7a960e5 diff --git a/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc b/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc index 1ad92784d47ac..85b491240e711 100644 --- a/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc +++ b/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc @@ -1 +1 @@ -not a - Logical not \ No newline at end of file +not a - Logical not diff --git a/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 b/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 index 6520d74f02566..0956574686405 100644 --- a/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 +++ b/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 @@ -1,2 +1,2 @@ not a - Logical not -Synonyms: ! \ No newline at end of file +Synonyms: ! diff --git a/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 b/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 index c0e84e95d01c2..e6b0fa3967745 100644 --- a/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 +++ b/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 @@ -1 +1 @@ -! a - Logical not \ No newline at end of file +! a - Logical not diff --git a/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a b/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a index 05cb1f6ab03d7..00f749069b4a3 100644 --- a/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a +++ b/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a @@ -1,2 +1,2 @@ ! a - Logical not -Synonyms: not \ No newline at end of file +Synonyms: not diff --git a/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e b/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e deleted file mode 100644 index 9e2108ff72b18..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e +++ /dev/null @@ -1 +0,0 @@ -a <> b - Returns TRUE if a is not equal to b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_notequal-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e rename to sql/hive/src/test/resources/golden/udf_notequal-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e b/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e new file mode 100644 index 0000000000000..5bcfd313a8ff6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e @@ -0,0 +1 @@ +a <> b - Returns TRUE if a is not equal to b diff --git a/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a b/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a deleted file mode 100644 index 162f808ed5b67..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a +++ /dev/null @@ -1,2 +0,0 @@ -a <> b - Returns TRUE if a is not equal to b -Synonyms: != \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 b/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 deleted file mode 100644 index 2dab01a3b2765..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 +++ /dev/null @@ -1 +0,0 @@ -a != b - Returns TRUE if a is not equal to b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a b/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a new file mode 100644 index 0000000000000..d8be357a858c4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a @@ -0,0 +1,2 @@ +a <> b - Returns TRUE if a is not equal to b +Synonyms: != diff --git a/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 b/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 new file mode 100644 index 0000000000000..aeffc394679cf --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 @@ -0,0 +1 @@ +a != b - Returns TRUE if a is not equal to b diff --git a/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 b/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 deleted file mode 100644 index 1f73486ba2896..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 +++ /dev/null @@ -1,2 +0,0 @@ -a != b - Returns TRUE if a is not equal to b -Synonyms: <> \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 b/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 new file mode 100644 index 0000000000000..32e73741d4cbc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 @@ -0,0 +1,2 @@ +a != b - Returns TRUE if a is not equal to b +Synonyms: <> diff --git a/sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 b/sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 b/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 deleted file mode 100644 index f82286d8bb069..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 +++ /dev/null @@ -1,499 +0,0 @@ -238 val_238 -86 val_86 -311 val_311 -27 val_27 -165 val_165 -409 val_409 -255 val_255 -278 val_278 -98 val_98 -484 val_484 -265 val_265 -193 val_193 -401 val_401 -150 val_150 -273 val_273 -224 val_224 -369 val_369 -66 val_66 -128 val_128 -213 val_213 -146 val_146 -406 val_406 -429 val_429 -374 val_374 -152 val_152 -469 val_469 -145 val_145 -495 val_495 -37 val_37 -327 val_327 -281 val_281 -277 val_277 -209 val_209 -15 val_15 -82 val_82 -403 val_403 -166 val_166 -417 val_417 -430 val_430 -252 val_252 -292 val_292 -219 val_219 -287 val_287 -153 val_153 -193 val_193 -338 val_338 -446 val_446 -459 val_459 -394 val_394 -237 val_237 -482 val_482 -174 val_174 -413 val_413 -494 val_494 -207 val_207 -199 val_199 -466 val_466 -208 val_208 -174 val_174 -399 val_399 -396 val_396 -247 val_247 -417 val_417 -489 val_489 -162 val_162 -377 val_377 -397 val_397 -309 val_309 -365 val_365 -266 val_266 -439 val_439 -342 val_342 -367 val_367 -325 val_325 -167 val_167 -195 val_195 -475 val_475 -17 val_17 -113 val_113 -155 val_155 -203 val_203 -339 val_339 -0 val_0 -455 val_455 -128 val_128 -311 val_311 -316 val_316 -57 val_57 -205 val_205 -149 val_149 -438 val_438 -345 val_345 -129 val_129 -170 val_170 -20 val_20 -489 val_489 -157 val_157 -378 val_378 -221 val_221 -92 val_92 -111 val_111 -47 val_47 -72 val_72 -4 val_4 -280 val_280 -35 val_35 -427 val_427 -277 val_277 -208 val_208 -356 val_356 -399 val_399 -169 val_169 -382 val_382 -498 val_498 -125 val_125 -386 val_386 -437 val_437 -469 val_469 -192 val_192 -286 val_286 -187 val_187 -176 val_176 -54 val_54 -459 val_459 -51 val_51 -138 val_138 -103 val_103 -239 val_239 -213 val_213 -216 val_216 -430 val_430 -278 val_278 -176 val_176 -289 val_289 -221 val_221 -65 val_65 -318 val_318 -332 val_332 -311 val_311 -275 val_275 -137 val_137 -241 val_241 -83 val_83 -333 val_333 -180 val_180 -284 val_284 -12 val_12 -230 val_230 -181 val_181 -67 val_67 -260 val_260 -404 val_404 -384 val_384 -489 val_489 -353 val_353 -373 val_373 -272 val_272 -138 val_138 -217 val_217 -84 val_84 -348 val_348 -466 val_466 -58 val_58 -8 val_8 -411 val_411 -230 val_230 -208 val_208 -348 val_348 -24 val_24 -463 val_463 -431 val_431 -179 val_179 -172 val_172 -42 val_42 -129 val_129 -158 val_158 -119 val_119 -496 val_496 -0 val_0 -322 val_322 -197 val_197 -468 val_468 -393 val_393 -454 val_454 -100 val_100 -298 val_298 -199 val_199 -191 val_191 -418 val_418 -96 val_96 -26 val_26 -165 val_165 -327 val_327 -230 val_230 -205 val_205 -120 val_120 -131 val_131 -51 val_51 -404 val_404 -43 val_43 -436 val_436 -156 val_156 -469 val_469 -468 val_468 -308 val_308 -95 val_95 -196 val_196 -288 val_288 -481 val_481 -457 val_457 -98 val_98 -282 val_282 -197 val_197 -187 val_187 -318 val_318 -318 val_318 -409 val_409 -470 val_470 -137 val_137 -369 val_369 -316 val_316 -169 val_169 -413 val_413 -85 val_85 -77 val_77 -0 val_0 -490 val_490 -87 val_87 -364 val_364 -179 val_179 -118 val_118 -134 val_134 -395 val_395 -282 val_282 -138 val_138 -238 val_238 -419 val_419 -15 val_15 -118 val_118 -72 val_72 -90 val_90 -307 val_307 -19 val_19 -435 val_435 -10 val_10 -277 val_277 -273 val_273 -306 val_306 -224 val_224 -309 val_309 -389 val_389 -327 val_327 -242 val_242 -369 val_369 -392 val_392 -272 val_272 -331 val_331 -401 val_401 -242 val_242 -452 val_452 -177 val_177 -226 val_226 -5 val_5 -497 val_497 -402 val_402 -396 val_396 -317 val_317 -395 val_395 -58 val_58 -35 val_35 -336 val_336 -95 val_95 -11 val_11 -168 val_168 -34 val_34 -229 val_229 -233 val_233 -143 val_143 -472 val_472 -322 val_322 -498 val_498 -160 val_160 -195 val_195 -42 val_42 -321 val_321 -430 val_430 -119 val_119 -489 val_489 -458 val_458 -78 val_78 -76 val_76 -41 val_41 -223 val_223 -492 val_492 -149 val_149 -449 val_449 -218 val_218 -228 val_228 -138 val_138 -453 val_453 -30 val_30 -209 val_209 -64 val_64 -468 val_468 -76 val_76 -74 val_74 -342 val_342 -69 val_69 -230 val_230 -33 val_33 -368 val_368 -103 val_103 -296 val_296 -113 val_113 -216 val_216 -367 val_367 -344 val_344 -167 val_167 -274 val_274 -219 val_219 -239 val_239 -485 val_485 -116 val_116 -223 val_223 -256 val_256 -263 val_263 -70 val_70 -487 val_487 -480 val_480 -401 val_401 -288 val_288 -191 val_191 -5 val_5 -244 val_244 -438 val_438 -128 val_128 -467 val_467 -432 val_432 -202 val_202 -316 val_316 -229 val_229 -469 val_469 -463 val_463 -280 val_280 -2 val_2 -35 val_35 -283 val_283 -331 val_331 -235 val_235 -80 val_80 -44 val_44 -193 val_193 -321 val_321 -335 val_335 -104 val_104 -466 val_466 -366 val_366 -175 val_175 -403 val_403 -483 val_483 -53 val_53 -105 val_105 -257 val_257 -406 val_406 -409 val_409 -190 val_190 -406 val_406 -401 val_401 -114 val_114 -258 val_258 -90 val_90 -203 val_203 -262 val_262 -348 val_348 -424 val_424 -12 val_12 -396 val_396 -201 val_201 -217 val_217 -164 val_164 -431 val_431 -454 val_454 -478 val_478 -298 val_298 -125 val_125 -431 val_431 -164 val_164 -424 val_424 -187 val_187 -382 val_382 -5 val_5 -70 val_70 -397 val_397 -480 val_480 -291 val_291 -24 val_24 -351 val_351 -255 val_255 -104 val_104 -70 val_70 -163 val_163 -438 val_438 -119 val_119 -414 val_414 -200 val_200 -491 val_491 -237 val_237 -439 val_439 -360 val_360 -248 val_248 -479 val_479 -305 val_305 -417 val_417 -199 val_199 -444 val_444 -120 val_120 -429 val_429 -169 val_169 -443 val_443 -323 val_323 -325 val_325 -277 val_277 -230 val_230 -478 val_478 -178 val_178 -468 val_468 -310 val_310 -317 val_317 -333 val_333 -493 val_493 -460 val_460 -207 val_207 -249 val_249 -265 val_265 -480 val_480 -83 val_83 -136 val_136 -353 val_353 -172 val_172 -214 val_214 -462 val_462 -233 val_233 -406 val_406 -133 val_133 -175 val_175 -189 val_189 -454 val_454 -375 val_375 -401 val_401 -421 val_421 -407 val_407 -384 val_384 -256 val_256 -26 val_26 -134 val_134 -67 val_67 -384 val_384 -379 val_379 -18 val_18 -462 val_462 -492 val_492 -100 val_100 -298 val_298 -9 val_9 -341 val_341 -498 val_498 -146 val_146 -458 val_458 -362 val_362 -186 val_186 -285 val_285 -348 val_348 -167 val_167 -18 val_18 -273 val_273 -183 val_183 -281 val_281 -344 val_344 -97 val_97 -469 val_469 -315 val_315 -84 val_84 -28 val_28 -37 val_37 -448 val_448 -152 val_152 -348 val_348 -307 val_307 -194 val_194 -414 val_414 -477 val_477 -222 val_222 -126 val_126 -90 val_90 -169 val_169 -403 val_403 -400 val_400 -200 val_200 -97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 b/sql/hive/src/test/resources/golden/udf_notequal-5-eb04e5ee00967799c913e8a5b424a332 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 rename to sql/hive/src/test/resources/golden/udf_notequal-5-eb04e5ee00967799c913e8a5b424a332 diff --git a/sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 b/sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 b/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 new file mode 100644 index 0000000000000..36244b327dc2f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 @@ -0,0 +1,499 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f b/sql/hive/src/test/resources/golden/udf_notequal-7-46a6514f2d7e6a097035ec1559df0096 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f rename to sql/hive/src/test/resources/golden/udf_notequal-7-46a6514f2d7e6a097035ec1559df0096 diff --git a/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd b/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd deleted file mode 100644 index f82286d8bb069..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd +++ /dev/null @@ -1,499 +0,0 @@ -238 val_238 -86 val_86 -311 val_311 -27 val_27 -165 val_165 -409 val_409 -255 val_255 -278 val_278 -98 val_98 -484 val_484 -265 val_265 -193 val_193 -401 val_401 -150 val_150 -273 val_273 -224 val_224 -369 val_369 -66 val_66 -128 val_128 -213 val_213 -146 val_146 -406 val_406 -429 val_429 -374 val_374 -152 val_152 -469 val_469 -145 val_145 -495 val_495 -37 val_37 -327 val_327 -281 val_281 -277 val_277 -209 val_209 -15 val_15 -82 val_82 -403 val_403 -166 val_166 -417 val_417 -430 val_430 -252 val_252 -292 val_292 -219 val_219 -287 val_287 -153 val_153 -193 val_193 -338 val_338 -446 val_446 -459 val_459 -394 val_394 -237 val_237 -482 val_482 -174 val_174 -413 val_413 -494 val_494 -207 val_207 -199 val_199 -466 val_466 -208 val_208 -174 val_174 -399 val_399 -396 val_396 -247 val_247 -417 val_417 -489 val_489 -162 val_162 -377 val_377 -397 val_397 -309 val_309 -365 val_365 -266 val_266 -439 val_439 -342 val_342 -367 val_367 -325 val_325 -167 val_167 -195 val_195 -475 val_475 -17 val_17 -113 val_113 -155 val_155 -203 val_203 -339 val_339 -0 val_0 -455 val_455 -128 val_128 -311 val_311 -316 val_316 -57 val_57 -205 val_205 -149 val_149 -438 val_438 -345 val_345 -129 val_129 -170 val_170 -20 val_20 -489 val_489 -157 val_157 -378 val_378 -221 val_221 -92 val_92 -111 val_111 -47 val_47 -72 val_72 -4 val_4 -280 val_280 -35 val_35 -427 val_427 -277 val_277 -208 val_208 -356 val_356 -399 val_399 -169 val_169 -382 val_382 -498 val_498 -125 val_125 -386 val_386 -437 val_437 -469 val_469 -192 val_192 -286 val_286 -187 val_187 -176 val_176 -54 val_54 -459 val_459 -51 val_51 -138 val_138 -103 val_103 -239 val_239 -213 val_213 -216 val_216 -430 val_430 -278 val_278 -176 val_176 -289 val_289 -221 val_221 -65 val_65 -318 val_318 -332 val_332 -311 val_311 -275 val_275 -137 val_137 -241 val_241 -83 val_83 -333 val_333 -180 val_180 -284 val_284 -12 val_12 -230 val_230 -181 val_181 -67 val_67 -260 val_260 -404 val_404 -384 val_384 -489 val_489 -353 val_353 -373 val_373 -272 val_272 -138 val_138 -217 val_217 -84 val_84 -348 val_348 -466 val_466 -58 val_58 -8 val_8 -411 val_411 -230 val_230 -208 val_208 -348 val_348 -24 val_24 -463 val_463 -431 val_431 -179 val_179 -172 val_172 -42 val_42 -129 val_129 -158 val_158 -119 val_119 -496 val_496 -0 val_0 -322 val_322 -197 val_197 -468 val_468 -393 val_393 -454 val_454 -100 val_100 -298 val_298 -199 val_199 -191 val_191 -418 val_418 -96 val_96 -26 val_26 -165 val_165 -327 val_327 -230 val_230 -205 val_205 -120 val_120 -131 val_131 -51 val_51 -404 val_404 -43 val_43 -436 val_436 -156 val_156 -469 val_469 -468 val_468 -308 val_308 -95 val_95 -196 val_196 -288 val_288 -481 val_481 -457 val_457 -98 val_98 -282 val_282 -197 val_197 -187 val_187 -318 val_318 -318 val_318 -409 val_409 -470 val_470 -137 val_137 -369 val_369 -316 val_316 -169 val_169 -413 val_413 -85 val_85 -77 val_77 -0 val_0 -490 val_490 -87 val_87 -364 val_364 -179 val_179 -118 val_118 -134 val_134 -395 val_395 -282 val_282 -138 val_138 -238 val_238 -419 val_419 -15 val_15 -118 val_118 -72 val_72 -90 val_90 -307 val_307 -19 val_19 -435 val_435 -10 val_10 -277 val_277 -273 val_273 -306 val_306 -224 val_224 -309 val_309 -389 val_389 -327 val_327 -242 val_242 -369 val_369 -392 val_392 -272 val_272 -331 val_331 -401 val_401 -242 val_242 -452 val_452 -177 val_177 -226 val_226 -5 val_5 -497 val_497 -402 val_402 -396 val_396 -317 val_317 -395 val_395 -58 val_58 -35 val_35 -336 val_336 -95 val_95 -11 val_11 -168 val_168 -34 val_34 -229 val_229 -233 val_233 -143 val_143 -472 val_472 -322 val_322 -498 val_498 -160 val_160 -195 val_195 -42 val_42 -321 val_321 -430 val_430 -119 val_119 -489 val_489 -458 val_458 -78 val_78 -76 val_76 -41 val_41 -223 val_223 -492 val_492 -149 val_149 -449 val_449 -218 val_218 -228 val_228 -138 val_138 -453 val_453 -30 val_30 -209 val_209 -64 val_64 -468 val_468 -76 val_76 -74 val_74 -342 val_342 -69 val_69 -230 val_230 -33 val_33 -368 val_368 -103 val_103 -296 val_296 -113 val_113 -216 val_216 -367 val_367 -344 val_344 -167 val_167 -274 val_274 -219 val_219 -239 val_239 -485 val_485 -116 val_116 -223 val_223 -256 val_256 -263 val_263 -70 val_70 -487 val_487 -480 val_480 -401 val_401 -288 val_288 -191 val_191 -5 val_5 -244 val_244 -438 val_438 -128 val_128 -467 val_467 -432 val_432 -202 val_202 -316 val_316 -229 val_229 -469 val_469 -463 val_463 -280 val_280 -2 val_2 -35 val_35 -283 val_283 -331 val_331 -235 val_235 -80 val_80 -44 val_44 -193 val_193 -321 val_321 -335 val_335 -104 val_104 -466 val_466 -366 val_366 -175 val_175 -403 val_403 -483 val_483 -53 val_53 -105 val_105 -257 val_257 -406 val_406 -409 val_409 -190 val_190 -406 val_406 -401 val_401 -114 val_114 -258 val_258 -90 val_90 -203 val_203 -262 val_262 -348 val_348 -424 val_424 -12 val_12 -396 val_396 -201 val_201 -217 val_217 -164 val_164 -431 val_431 -454 val_454 -478 val_478 -298 val_298 -125 val_125 -431 val_431 -164 val_164 -424 val_424 -187 val_187 -382 val_382 -5 val_5 -70 val_70 -397 val_397 -480 val_480 -291 val_291 -24 val_24 -351 val_351 -255 val_255 -104 val_104 -70 val_70 -163 val_163 -438 val_438 -119 val_119 -414 val_414 -200 val_200 -491 val_491 -237 val_237 -439 val_439 -360 val_360 -248 val_248 -479 val_479 -305 val_305 -417 val_417 -199 val_199 -444 val_444 -120 val_120 -429 val_429 -169 val_169 -443 val_443 -323 val_323 -325 val_325 -277 val_277 -230 val_230 -478 val_478 -178 val_178 -468 val_468 -310 val_310 -317 val_317 -333 val_333 -493 val_493 -460 val_460 -207 val_207 -249 val_249 -265 val_265 -480 val_480 -83 val_83 -136 val_136 -353 val_353 -172 val_172 -214 val_214 -462 val_462 -233 val_233 -406 val_406 -133 val_133 -175 val_175 -189 val_189 -454 val_454 -375 val_375 -401 val_401 -421 val_421 -407 val_407 -384 val_384 -256 val_256 -26 val_26 -134 val_134 -67 val_67 -384 val_384 -379 val_379 -18 val_18 -462 val_462 -492 val_492 -100 val_100 -298 val_298 -9 val_9 -341 val_341 -498 val_498 -146 val_146 -458 val_458 -362 val_362 -186 val_186 -285 val_285 -348 val_348 -167 val_167 -18 val_18 -273 val_273 -183 val_183 -281 val_281 -344 val_344 -97 val_97 -469 val_469 -315 val_315 -84 val_84 -28 val_28 -37 val_37 -448 val_448 -152 val_152 -348 val_348 -307 val_307 -194 val_194 -414 val_414 -477 val_477 -222 val_222 -126 val_126 -90 val_90 -169 val_169 -403 val_403 -400 val_400 -200 val_200 -97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd b/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd new file mode 100644 index 0000000000000..36244b327dc2f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd @@ -0,0 +1,499 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 b/sql/hive/src/test/resources/golden/udf_notop-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 rename to sql/hive/src/test/resources/golden/udf_notop-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 b/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 deleted file mode 100644 index a55e3339049e8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 +++ /dev/null @@ -1 +0,0 @@ -false true false false true false true false true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 b/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 new file mode 100644 index 0000000000000..5ffd61b380318 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 @@ -0,0 +1 @@ +false true false false true false true false true true diff --git a/sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_nvl-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_nvl-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 b/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 deleted file mode 100644 index 5ffcb851d56f9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 +++ /dev/null @@ -1 +0,0 @@ -nvl(value,default_value) - Returns default value if value is null else returns value \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 b/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 deleted file mode 100644 index 5afff220da695..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 +++ /dev/null @@ -1,4 +0,0 @@ -nvl(value,default_value) - Returns default value if value is null else returns value -Example: - > SELECT nvl(null,'bla') FROM src LIMIT 1; - bla \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 b/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 new file mode 100644 index 0000000000000..b043150b9c901 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 @@ -0,0 +1 @@ +nvl(value,default_value) - Returns default value if value is null else returns value diff --git a/sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def b/sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 b/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 new file mode 100644 index 0000000000000..4daa1bbffa621 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 @@ -0,0 +1,4 @@ +nvl(value,default_value) - Returns default value if value is null else returns value +Example: + > SELECT nvl(null,'bla') FROM src LIMIT 1; + bla diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef b/sql/hive/src/test/resources/golden/udf_nvl-3-47199a1c23cb1cc6827c601bb66513d3 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef rename to sql/hive/src/test/resources/golden/udf_nvl-3-47199a1c23cb1cc6827c601bb66513d3 diff --git a/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 b/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 deleted file mode 100644 index 2087e17494459..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 +++ /dev/null @@ -1 +0,0 @@ -1 5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 b/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 new file mode 100644 index 0000000000000..273bc7331072a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 @@ -0,0 +1 @@ +1 5 diff --git a/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 b/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 index 9138f44ad2a41..f0e20ea40509c 100644 --- a/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 +++ b/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 @@ -1 +1 @@ -a or b - Logical or \ No newline at end of file +a or b - Logical or diff --git a/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 b/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 index 9138f44ad2a41..f0e20ea40509c 100644 --- a/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 +++ b/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 @@ -1 +1 @@ -a or b - Logical or \ No newline at end of file +a or b - Logical or diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 b/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 index 0535b085e50c2..1ce92fd715822 100644 --- a/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 +++ b/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 @@ -1 +1 @@ -parse_url(url, partToExtract[, key]) - extracts a part from a URL \ No newline at end of file +parse_url(url, partToExtract[, key]) - extracts a part from a URL diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab b/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab index 7178e07a07e48..bd448a4e4cade 100644 --- a/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab +++ b/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab @@ -7,4 +7,4 @@ Example: > SELECT parse_url('http://facebook.com/path/p1.php?query=1', 'QUERY') FROM src LIMIT 1; 'query=1' > SELECT parse_url('http://facebook.com/path/p1.php?query=1', 'QUERY', 'query') FROM src LIMIT 1; - '1' \ No newline at end of file + '1' diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 b/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 index f5483d4e3dafe..e3ece483b53fd 100644 --- a/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 +++ b/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 @@ -1 +1 @@ -facebook.com /path1/p.php k1=v1&k2=v2 Ref1 v2 v1 NULL /path1/p.php?k1=v1&k2=v2 http NULL facebook.com \ No newline at end of file +facebook.com /path1/p.php k1=v1&k2=v2 Ref1 v2 v1 NULL /path1/p.php?k1=v1&k2=v2 http NULL facebook.com diff --git a/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 b/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 deleted file mode 100644 index 2025042f5d493..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 +++ /dev/null @@ -1 +0,0 @@ -percentile(expr, pc) - Returns the percentile(s) of expr at pc (range: [0,1]).pc can be a double or double array diff --git a/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a b/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a deleted file mode 100644 index 2025042f5d493..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a +++ /dev/null @@ -1 +0,0 @@ -percentile(expr, pc) - Returns the percentile(s) of expr at pc (range: [0,1]).pc can be a double or double array diff --git a/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 deleted file mode 100644 index 5d2fc352ee060..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 +++ /dev/null @@ -1 +0,0 @@ -a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae deleted file mode 100644 index 5d2fc352ee060..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae +++ /dev/null @@ -1 +0,0 @@ -a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 deleted file mode 100644 index 0b46af11c4516..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 +++ /dev/null @@ -1 +0,0 @@ -6.89 51.7 18.09 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 deleted file mode 100644 index 5eb0813b60eb6..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 +++ /dev/null @@ -1 +0,0 @@ -8 51 15 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 deleted file mode 100644 index e21e4b08e7a62..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 +++ /dev/null @@ -1 +0,0 @@ -5 50 0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 deleted file mode 100644 index e0bc2a844fb46..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 +++ /dev/null @@ -1 +0,0 @@ -8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 deleted file mode 100644 index e0bc2a844fb46..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 +++ /dev/null @@ -1 +0,0 @@ -8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 deleted file mode 100644 index e0bc2a844fb46..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 +++ /dev/null @@ -1 +0,0 @@ -8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd deleted file mode 100644 index 48371142e9b5d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd +++ /dev/null @@ -1 +0,0 @@ -6.8899984 51.700005 18.089996 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 deleted file mode 100644 index ab842acd48b3c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 +++ /dev/null @@ -1 +0,0 @@ -6.890000000000011 51.699999999999996 18.090000000000003 diff --git a/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 b/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 index 7c98729cc195c..6374b08a607ab 100644 --- a/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 +++ b/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 @@ -1 +1 @@ -positive a - Returns a \ No newline at end of file +positive a - Returns a diff --git a/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c b/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c index 7c98729cc195c..6374b08a607ab 100644 --- a/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c +++ b/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c @@ -1 +1 @@ -positive a - Returns a \ No newline at end of file +positive a - Returns a diff --git a/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 b/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 index e8f11c444a808..9d794539b4ae0 100644 --- a/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 +++ b/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 @@ -1 +1 @@ -a + b - Returns a+b \ No newline at end of file +a + b - Returns a+b diff --git a/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 b/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 index e8f11c444a808..9d794539b4ae0 100644 --- a/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 +++ b/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 @@ -1 +1 @@ -a + b - Returns a+b \ No newline at end of file +a + b - Returns a+b diff --git a/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 b/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 index 43f197e360c82..67377963d2aa1 100644 --- a/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 +++ b/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 @@ -1 +1 @@ -pow(x1, x2) - raise x1 to the power of x2 \ No newline at end of file +pow(x1, x2) - raise x1 to the power of x2 diff --git a/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 b/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 index afc2e4462a1a8..ded9d10a595ff 100644 --- a/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 +++ b/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 @@ -2,4 +2,4 @@ pow(x1, x2) - raise x1 to the power of x2 Synonyms: power Example: > SELECT pow(2, 3) FROM src LIMIT 1; - 8 \ No newline at end of file + 8 diff --git a/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 b/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 index 5e3a6a8f31fd3..90e23c3255b77 100644 --- a/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 +++ b/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 @@ -1 +1 @@ -power(x1, x2) - raise x1 to the power of x2 \ No newline at end of file +power(x1, x2) - raise x1 to the power of x2 diff --git a/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 b/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 index c3414e29768e3..4890e2e989d34 100644 --- a/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 +++ b/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 @@ -2,4 +2,4 @@ power(x1, x2) - raise x1 to the power of x2 Synonyms: pow Example: > SELECT power(2, 3) FROM src LIMIT 1; - 8 \ No newline at end of file + 8 diff --git a/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e b/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e deleted file mode 100644 index 1635ff88dd768..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e +++ /dev/null @@ -1 +0,0 @@ -printf(String format, Obj... args) - function that can format strings according to printf-style format strings diff --git a/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee b/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee deleted file mode 100644 index 62440ee68e145..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee +++ /dev/null @@ -1,4 +0,0 @@ -printf(String format, Obj... args) - function that can format strings according to printf-style format strings -Example: - > SELECT printf("Hello World %d %s", 100, "days")FROM src LIMIT 1; - "Hello World 100 days" diff --git a/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 b/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 b/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 deleted file mode 100644 index 39cb945991403..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 +++ /dev/null @@ -1 +0,0 @@ -Hello World 100 days diff --git a/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb b/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb deleted file mode 100644 index 04bf5e552a576..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb +++ /dev/null @@ -1 +0,0 @@ -All Type Test: false, A, 15000, 1.234000e+01, +27183.2401, 2300.41, 32, corret, 0x1.002p8 diff --git a/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 b/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 deleted file mode 100644 index 2e9f7509968a3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 +++ /dev/null @@ -1 +0,0 @@ -Color red, String Null: null, number1 123456, number2 00089, Integer Null: null, hex 0xff, float 3.14 Double Null: null diff --git a/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 b/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_radians-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_radians-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 b/sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 b/sql/hive/src/test/resources/golden/udf_radians-1-58b73fc96927d447d1225f021eaa378 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 rename to sql/hive/src/test/resources/golden/udf_radians-1-58b73fc96927d447d1225f021eaa378 diff --git a/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 b/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 deleted file mode 100644 index 116b6bc461ed0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 +++ /dev/null @@ -1 +0,0 @@ -1.000000357564167 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 new file mode 100644 index 0000000000000..b00bf83e61d82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 @@ -0,0 +1,5 @@ +radians(x) - Converts degrees to radians +Example: + > SELECT radians(90) FROM src LIMIT 1; + 1.5707963267949mo + diff --git a/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 b/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 deleted file mode 100644 index aaf3b31fef488..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 +++ /dev/null @@ -1 +0,0 @@ -2.4999991485811655 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 b/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 new file mode 100644 index 0000000000000..6b0996864478b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 @@ -0,0 +1 @@ +1.000000357564167 diff --git a/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 deleted file mode 100644 index 73ad88be4ef3d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 +++ /dev/null @@ -1 +0,0 @@ -radians(x) - Converts degrees to radians \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 b/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 new file mode 100644 index 0000000000000..1f204866982ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 @@ -0,0 +1 @@ +2.4999991485811655 diff --git a/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 new file mode 100644 index 0000000000000..99e71b16f18ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 @@ -0,0 +1 @@ +radians(x) - Converts degrees to radians diff --git a/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 deleted file mode 100644 index e0237c0058f55..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 +++ /dev/null @@ -1,4 +0,0 @@ -radians(x) - Converts degrees to radians -Example: - > SELECT radians(90) FROM src LIMIT 1; - 1.5707963267949mo diff --git a/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 new file mode 100644 index 0000000000000..b00bf83e61d82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 @@ -0,0 +1,5 @@ +radians(x) - Converts degrees to radians +Example: + > SELECT radians(90) FROM src LIMIT 1; + 1.5707963267949mo + diff --git a/sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 b/sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 b/sql/hive/src/test/resources/golden/udf_radians-6-70c9e7199b5898e2c3a4943ec58da113 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 rename to sql/hive/src/test/resources/golden/udf_radians-6-70c9e7199b5898e2c3a4943ec58da113 diff --git a/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 b/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 deleted file mode 100644 index 116b6bc461ed0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 +++ /dev/null @@ -1 +0,0 @@ -1.000000357564167 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 b/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 deleted file mode 100644 index aaf3b31fef488..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 +++ /dev/null @@ -1 +0,0 @@ -2.4999991485811655 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 b/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 new file mode 100644 index 0000000000000..6b0996864478b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 @@ -0,0 +1 @@ +1.000000357564167 diff --git a/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 deleted file mode 100644 index 73ad88be4ef3d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 +++ /dev/null @@ -1 +0,0 @@ -radians(x) - Converts degrees to radians \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 b/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 new file mode 100644 index 0000000000000..1f204866982ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 @@ -0,0 +1 @@ +2.4999991485811655 diff --git a/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 new file mode 100644 index 0000000000000..99e71b16f18ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 @@ -0,0 +1 @@ +radians(x) - Converts degrees to radians diff --git a/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 deleted file mode 100644 index e0237c0058f55..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 +++ /dev/null @@ -1,4 +0,0 @@ -radians(x) - Converts degrees to radians -Example: - > SELECT radians(90) FROM src LIMIT 1; - 1.5707963267949mo diff --git a/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 b/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 index d19c5b50fe5c4..a4de35428cc8f 100644 --- a/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 +++ b/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 @@ -1 +1 @@ -rand([seed]) - Returns a pseudorandom number between 0 and 1 \ No newline at end of file +rand([seed]) - Returns a pseudorandom number between 0 and 1 diff --git a/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e b/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e index d19c5b50fe5c4..a4de35428cc8f 100644 --- a/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e +++ b/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e @@ -1 +1 @@ -rand([seed]) - Returns a pseudorandom number between 0 and 1 \ No newline at end of file +rand([seed]) - Returns a pseudorandom number between 0 and 1 diff --git a/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 b/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 deleted file mode 100644 index 1d7658151cd62..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 +++ /dev/null @@ -1 +0,0 @@ -reflect(class,method[,arg1[,arg2..]]) calls method with reflection diff --git a/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee b/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee deleted file mode 100644 index ddf986c01e5b1..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee +++ /dev/null @@ -1,4 +0,0 @@ -reflect(class,method[,arg1[,arg2..]]) calls method with reflection -Synonyms: java_method -Use this UDF to call Java methods by matching the argument signature - diff --git a/sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 b/sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 b/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 deleted file mode 100644 index cd35e5b290db5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 +++ /dev/null @@ -1 +0,0 @@ -reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 b/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 deleted file mode 100644 index 48ef97292ab62..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 +++ /dev/null @@ -1,3 +0,0 @@ -reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection -Use this UDF to call Java methods by matching the argument signature - diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca b/sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 b/sql/hive/src/test/resources/golden/udf_regexp-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 rename to sql/hive/src/test/resources/golden/udf_regexp-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 b/sql/hive/src/test/resources/golden/udf_regexp-1-19917611f74aedc0922560f7f2595948 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 rename to sql/hive/src/test/resources/golden/udf_regexp-1-19917611f74aedc0922560f7f2595948 diff --git a/sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 b/sql/hive/src/test/resources/golden/udf_regexp-2-f7f0527cd47612d7f256edd5f8963800 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 rename to sql/hive/src/test/resources/golden/udf_regexp-2-f7f0527cd47612d7f256edd5f8963800 diff --git a/sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 b/sql/hive/src/test/resources/golden/udf_regexp-3-59aff54bae544ee620141e4e629f167a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 rename to sql/hive/src/test/resources/golden/udf_regexp-3-59aff54bae544ee620141e4e629f167a diff --git a/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 b/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 index 429057caf71f0..6f4b3cea94c03 100644 --- a/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 +++ b/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 @@ -1 +1 @@ -regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp \ No newline at end of file +regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp diff --git a/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb b/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb index 30e237490811f..fc9d4aa686db9 100644 --- a/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb +++ b/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb @@ -1,4 +1,4 @@ regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp Example: > SELECT regexp_extract('100-200', '(\d+)-(\d+)', 1) FROM src LIMIT 1; - '100' \ No newline at end of file + '100' diff --git a/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e b/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e index 1a38701d68875..193093b0edcaa 100644 --- a/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e +++ b/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e @@ -1 +1 @@ -regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep \ No newline at end of file +regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep diff --git a/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc b/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc index bebfabaf637fb..6e3577aba5da2 100644 --- a/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc +++ b/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc @@ -1,4 +1,4 @@ regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep Example: > SELECT regexp_replace('100-200', '(\d+)', 'num') FROM src LIMIT 1; - 'num-num' \ No newline at end of file + 'num-num' diff --git a/sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_repeat-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_repeat-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b b/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b deleted file mode 100644 index 694c367436f3c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b +++ /dev/null @@ -1 +0,0 @@ -repeat(str, n) - repeat str n times \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b b/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b deleted file mode 100644 index 5df19ba1c2cbf..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b +++ /dev/null @@ -1,4 +0,0 @@ -repeat(str, n) - repeat str n times -Example: - > SELECT repeat('123', 2) FROM src LIMIT 1; - '123123' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b b/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b new file mode 100644 index 0000000000000..23a6a30503468 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b @@ -0,0 +1 @@ +repeat(str, n) - repeat str n times diff --git a/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b b/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b new file mode 100644 index 0000000000000..4f4b491b2807c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b @@ -0,0 +1,4 @@ +repeat(str, n) - repeat str n times +Example: + > SELECT repeat('123', 2) FROM src LIMIT 1; + '123123' diff --git a/sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a b/sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d b/sql/hive/src/test/resources/golden/udf_repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d rename to sql/hive/src/test/resources/golden/udf_repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec diff --git a/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 b/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 deleted file mode 100644 index 45425cf087c09..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 +++ /dev/null @@ -1 +0,0 @@ -FacebookFacebookFacebook \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 b/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 new file mode 100644 index 0000000000000..5a355c1c58fc7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 @@ -0,0 +1 @@ +FacebookFacebookFacebook diff --git a/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 b/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 index fd9ac3081a1cc..068aeb36f6477 100644 --- a/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 +++ b/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 @@ -1 +1 @@ -str rlike regexp - Returns true if str matches regexp and false otherwise \ No newline at end of file +str rlike regexp - Returns true if str matches regexp and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca b/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca index 43b06945caa54..ab346793b9d3d 100644 --- a/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca +++ b/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca @@ -2,4 +2,4 @@ str rlike regexp - Returns true if str matches regexp and false otherwise Synonyms: regexp Example: > SELECT 'fb' rlike '.*' FROM src LIMIT 1; - true \ No newline at end of file + true diff --git a/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 deleted file mode 100644 index e4586b2e73a93..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 +++ /dev/null @@ -1 +0,0 @@ -round(x[, d]) - round x to d decimal places \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_round-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_round-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 new file mode 100644 index 0000000000000..49fdc0a774e70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 @@ -0,0 +1 @@ +round(x[, d]) - round x to d decimal places diff --git a/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 deleted file mode 100644 index c0d5b480e9751..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 +++ /dev/null @@ -1,4 +0,0 @@ -round(x[, d]) - round x to d decimal places -Example: - > SELECT round(12.3456, 1) FROM src LIMIT 1; - 12.3' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 new file mode 100644 index 0000000000000..862adeae821ff --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 @@ -0,0 +1,4 @@ +round(x[, d]) - round x to d decimal places +Example: + > SELECT round(12.3456, 1) FROM src LIMIT 1; + 12.3' diff --git a/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 b/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 deleted file mode 100644 index 0924e3869076d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL Infinity NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c b/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c deleted file mode 100644 index 3b9c30929a240..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c +++ /dev/null @@ -1 +0,0 @@ -55555 55555.0 55555.0 55555.0 55555.0 55560.0 55600.0 56000.0 60000.0 100000.0 0.0 0.0 0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de b/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de new file mode 100644 index 0000000000000..fc6e4224259d8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de @@ -0,0 +1 @@ +NULL NULL NULL NULL NaN diff --git a/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 b/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 new file mode 100644 index 0000000000000..f8d833cc0880e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 @@ -0,0 +1 @@ +55555 55555 55555 55555 55555 55560 55600 56000 60000 100000 0 0 0 diff --git a/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 b/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 deleted file mode 100644 index 1f243f6cbc6db..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 +++ /dev/null @@ -1 +0,0 @@ -125.0 125.0 125.3 125.32 125.315 125.315 130.0 100.0 0.0 0.0 -125.0 -125.0 -125.3 -125.32 -125.315 -125.315 -130.0 -100.0 0.0 0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d b/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d new file mode 100644 index 0000000000000..389ab6417f19d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d @@ -0,0 +1 @@ +125.0 125.0 125.3 125.32 125.315 125.315 130.0 100.0 0.0 0.0 -125.0 -125.0 -125.3 -125.32 -125.315 -125.315 -130.0 -100.0 0.0 0.0 diff --git a/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a b/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a deleted file mode 100644 index 918404cec8047..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a +++ /dev/null @@ -1 +0,0 @@ -0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 3.0 3.1 3.14 3.142 3.1416 3.14159 3.141593 3.1415927 3.14159265 3.141592654 3.1415926536 3.14159265359 3.14159265359 3.1415926535898 3.1415926535898 3.14159265358979 3.141592653589793 3.141592653589793 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 b/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 new file mode 100644 index 0000000000000..3b083b3e26c4a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 @@ -0,0 +1 @@ +0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 3.0 3.1 3.14 3.142 3.1416 3.14159 3.141593 3.1415927 3.14159265 3.141592654 3.1415926536 3.14159265359 3.14159265359 3.1415926535898 3.1415926535898 3.14159265358979 3.141592653589793 3.141592653589793 diff --git a/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b b/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b deleted file mode 100644 index af105563af144..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b +++ /dev/null @@ -1 +0,0 @@ -1809242.315111134 -1809242.315111134 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d b/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d new file mode 100644 index 0000000000000..c3496bf5b6d1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d @@ -0,0 +1 @@ +1809242.315111134 -1809242.315111134 1809242.315111134 -1809242.315111134 diff --git a/sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 b/sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce b/sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 b/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 b/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 b/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b b/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b deleted file mode 100644 index f40e633f703c9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b +++ /dev/null @@ -1 +0,0 @@ -Infinity Infinity Infinity Infinity \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/udf_round_3-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a rename to sql/hive/src/test/resources/golden/udf_round_3-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c b/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c deleted file mode 100644 index 3714de0db18dc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c +++ /dev/null @@ -1 +0,0 @@ --128 127 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d b/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d new file mode 100644 index 0000000000000..0a104d81ef51e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d @@ -0,0 +1 @@ +-128 127 0 diff --git a/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 b/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 deleted file mode 100644 index a9265d7b8a1b7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 +++ /dev/null @@ -1 +0,0 @@ --32768 32767 -129 128 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 b/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 deleted file mode 100644 index 2d25f54073df7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 +++ /dev/null @@ -1 +0,0 @@ --2147483648 2147483647 -32769 32768 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca b/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca new file mode 100644 index 0000000000000..972dee75fae8a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca @@ -0,0 +1 @@ +-32768 32767 -129 128 diff --git a/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 b/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 new file mode 100644 index 0000000000000..50928a309cf3a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 @@ -0,0 +1 @@ +-2147483648 2147483647 -32769 32768 diff --git a/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 b/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 deleted file mode 100644 index 41a2624c6cfeb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 +++ /dev/null @@ -1 +0,0 @@ --9223372036854775808 9223372036854775807 -2147483649 2147483648 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 b/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 new file mode 100644 index 0000000000000..c9e55e0ec0b95 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 @@ -0,0 +1 @@ +-9223372036854775808 9223372036854775807 -2147483649 2147483648 diff --git a/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 b/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 deleted file mode 100644 index 98d3f53dfc442..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 +++ /dev/null @@ -1 +0,0 @@ -126.0 127.0 32766.0 32767.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad b/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad new file mode 100644 index 0000000000000..3898a62cae1b5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad @@ -0,0 +1 @@ +126.0 127.0 32766.0 32767.0 diff --git a/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/udf_rpad-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 rename to sql/hive/src/test/resources/golden/udf_rpad-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c b/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c deleted file mode 100644 index 7cb2d71d4b80d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c +++ /dev/null @@ -1 +0,0 @@ -rpad(str, len, pad) - Returns str, right-padded with pad to a length of len \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 b/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 deleted file mode 100644 index 2b198b8dc96c4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 +++ /dev/null @@ -1,6 +0,0 @@ -rpad(str, len, pad) - Returns str, right-padded with pad to a length of len -If str is longer than len, the return value is shortened to len characters. -Example: - > SELECT rpad('hi', 5, '??') FROM src LIMIT 1; - 'hi???' > SELECT rpad('hi', 1, '??') FROM src LIMIT 1; - 'h' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c b/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c new file mode 100644 index 0000000000000..f451030a3a142 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c @@ -0,0 +1 @@ +rpad(str, len, pad) - Returns str, right-padded with pad to a length of len diff --git a/sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 b/sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 b/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 new file mode 100644 index 0000000000000..67dcf2427362d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 @@ -0,0 +1,6 @@ +rpad(str, len, pad) - Returns str, right-padded with pad to a length of len +If str is longer than len, the return value is shortened to len characters. +Example: + > SELECT rpad('hi', 5, '??') FROM src LIMIT 1; + 'hi???' > SELECT rpad('hi', 1, '??') FROM src LIMIT 1; + 'h' diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b b/sql/hive/src/test/resources/golden/udf_rpad-3-66acb969c28a8e376782ccd0d442b450 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b rename to sql/hive/src/test/resources/golden/udf_rpad-3-66acb969c28a8e376782ccd0d442b450 diff --git a/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 b/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 deleted file mode 100644 index b2645e365c8a9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 +++ /dev/null @@ -1 +0,0 @@ -h hi... hi1231 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b b/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b new file mode 100644 index 0000000000000..0d73ca82abf89 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b @@ -0,0 +1 @@ +h hi... hi1231 diff --git a/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a b/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a index 94d6aeed8f533..d0d2416d7ee90 100644 --- a/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a +++ b/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a @@ -1 +1 @@ -rtrim(str) - Removes the trailing space characters from str \ No newline at end of file +rtrim(str) - Removes the trailing space characters from str diff --git a/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd b/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd index 957e608a7c732..697cdcc8198b3 100644 --- a/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd +++ b/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd @@ -1,4 +1,4 @@ rtrim(str) - Removes the trailing space characters from str Example: > SELECT rtrim('facebook ') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/udf_second-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a rename to sql/hive/src/test/resources/golden/udf_second-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 b/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 deleted file mode 100644 index 577c90254cb5a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 +++ /dev/null @@ -1 +0,0 @@ -second(date) - Returns the second of date \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 b/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 deleted file mode 100644 index 4b48294e5b9ad..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 +++ /dev/null @@ -1,7 +0,0 @@ -second(date) - Returns the second of date -date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'. -Example: - > SELECT second('2009-07-30 12:58:59') FROM src LIMIT 1; - 59 - > SELECT second('12:58:59') FROM src LIMIT 1; - 59 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 b/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 new file mode 100644 index 0000000000000..ad2b24b8eb11c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 @@ -0,0 +1 @@ +second(date) - Returns the second of date diff --git a/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 b/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 new file mode 100644 index 0000000000000..9dc38c3e79129 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 @@ -0,0 +1,7 @@ +second(date) - Returns the second of date +date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'. +Example: + > SELECT second('2009-07-30 12:58:59') FROM src LIMIT 1; + 59 + > SELECT second('12:58:59') FROM src LIMIT 1; + 59 diff --git a/sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f b/sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf b/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf deleted file mode 100644 index 4d5ef5cf4a699..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf +++ /dev/null @@ -1 +0,0 @@ -15 15 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 b/sql/hive/src/test/resources/golden/udf_second-3-d678372e3837a16be245d2e33482f17f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 rename to sql/hive/src/test/resources/golden/udf_second-3-d678372e3837a16be245d2e33482f17f diff --git a/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf b/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf new file mode 100644 index 0000000000000..3ddfab9c754f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf @@ -0,0 +1 @@ +15 15 NULL diff --git a/sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f b/sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 b/sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 b/sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 b/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 deleted file mode 100644 index fea4f860c8465..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 +++ /dev/null @@ -1,31 +0,0 @@ -41626672616765 -446174656E -48697665 -496E74657270756E6B74696F6E -4D756C7469706C65 -53C3A4747A65 -554446 -5665727765636873656C756E67 -5765726B7A657567 -616C73 -61757367657A656963686E65746573 -646965 -646965 -6469657365 -646F6368 -65696E -66756E6B74696F6E69657274 -66C3BC72 -676562696C646574656E -696D6D6572 -697374 -697374 -6D61736368696E656C6C65 -6E6F6368 -7363686C65636874 -756E64 -756E64 -7669656C6C6569636874 -7669656C7365697469676572 -766F6E -C39C6265727365747A756E67 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 b/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 deleted file mode 100644 index c49ca6bb5a1b0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 +++ /dev/null @@ -1 +0,0 @@ -[["Hive","is","an","excellent","tool","for","data","querying","and","perhaps","more","versatile","than","machine","translation"],["Multiple","ill-formed","sentences","confounding","punctuation","and","yet","this","UDF","still","works"]] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 b/sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 b/sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 b/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 deleted file mode 100644 index b798628e56686..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 +++ /dev/null @@ -1,50 +0,0 @@ -48697665 -48697665 -554446 -6175746F6D617469717565 -6175746F6D617469717565 -6365 -636F6E667573696F6E -6465 -6465 -646573 -646F6E6EC3A96573 -646F6E6EC3A96573 -656E636F7265 -657374 -657374 -6574 -6574 -6574 -657863656C6C656E74 -657863656C6C656E74 -666F6E6374696F6E6E65 -666F726DC3A96573 -6C61 -6C61 -6C61 -6C6573 -6C6573 -6D616C -6D756C7469706C6573 -6F7574696C -6F7574696C -706575742DC3AA747265 -706575742DC3AA747265 -70687261736573 -706C7573 -706C7573 -706F6C7976616C656E74 -706F6C7976616C656E74 -706F6E6374756174696F6E -706F7572 -706F7572 -706F757274616E74 -717565 -717565 -72657175C3AA746573 -72657175C3AA746573 -74726164756374696F6E -74726164756374696F6E -756E -756E \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 b/sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de b/sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f b/sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 b/sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 b/sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e b/sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_sign-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_sign-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c b/sql/hive/src/test/resources/golden/udf_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c rename to sql/hive/src/test/resources/golden/udf_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9 diff --git a/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 b/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a b/sql/hive/src/test/resources/golden/udf_sign-10-9a5326b0bf612fed4ce0b04770bebc16 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a rename to sql/hive/src/test/resources/golden/udf_sign-10-9a5326b0bf612fed4ce0b04770bebc16 diff --git a/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa deleted file mode 100644 index 3345fbb8f2c35..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa +++ /dev/null @@ -1 +0,0 @@ -sign(x) - returns the sign of x ) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a deleted file mode 100644 index bdacec8810e77..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a +++ /dev/null @@ -1,4 +0,0 @@ -sign(x) - returns the sign of x ) -Example: - > SELECT sign(40) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa new file mode 100644 index 0000000000000..60533947bcfb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa @@ -0,0 +1 @@ +sign(x) - returns the sign of x ) diff --git a/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a new file mode 100644 index 0000000000000..d888f9e9cda19 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a @@ -0,0 +1,4 @@ +sign(x) - returns the sign of x ) +Example: + > SELECT sign(40) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef b/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef deleted file mode 100644 index 31a5b0b81dc51..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef +++ /dev/null @@ -1 +0,0 @@ --1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 b/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d b/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d deleted file mode 100644 index 9f8e9b69a33f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d +++ /dev/null @@ -1 +0,0 @@ -1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 b/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 new file mode 100644 index 0000000000000..18e16e38c5de6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 @@ -0,0 +1 @@ +-1.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c b/sql/hive/src/test/resources/golden/udf_sign-4-9a5326b0bf612fed4ce0b04770bebc16 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c rename to sql/hive/src/test/resources/golden/udf_sign-4-9a5326b0bf612fed4ce0b04770bebc16 diff --git a/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa deleted file mode 100644 index 3345fbb8f2c35..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa +++ /dev/null @@ -1 +0,0 @@ -sign(x) - returns the sign of x ) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a deleted file mode 100644 index bdacec8810e77..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a +++ /dev/null @@ -1,4 +0,0 @@ -sign(x) - returns the sign of x ) -Example: - > SELECT sign(40) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa new file mode 100644 index 0000000000000..60533947bcfb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa @@ -0,0 +1 @@ +sign(x) - returns the sign of x ) diff --git a/sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e b/sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a new file mode 100644 index 0000000000000..d888f9e9cda19 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a @@ -0,0 +1,4 @@ +sign(x) - returns the sign of x ) +Example: + > SELECT sign(40) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 b/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a b/sql/hive/src/test/resources/golden/udf_sign-7-ed2aaa1a416c0cccc04de970424e1860 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a rename to sql/hive/src/test/resources/golden/udf_sign-7-ed2aaa1a416c0cccc04de970424e1860 diff --git a/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef b/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef deleted file mode 100644 index 31a5b0b81dc51..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef +++ /dev/null @@ -1 +0,0 @@ --1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 b/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d b/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d deleted file mode 100644 index 9f8e9b69a33f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d +++ /dev/null @@ -1 +0,0 @@ -1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 b/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 new file mode 100644 index 0000000000000..18e16e38c5de6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 @@ -0,0 +1 @@ +-1.0 diff --git a/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee b/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee deleted file mode 100644 index 6155d9a2eae55..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee +++ /dev/null @@ -1 +0,0 @@ -sin(x) - returns the sine of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_sin-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_sin-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 b/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 deleted file mode 100644 index a39ed8840b916..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 +++ /dev/null @@ -1,4 +0,0 @@ -sin(x) - returns the sine of x (x is in radians) -Example: - > SELECT sin(0) FROM src LIMIT 1; - 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee b/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee new file mode 100644 index 0000000000000..86bbf99dbf067 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee @@ -0,0 +1 @@ +sin(x) - returns the sine of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 b/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 new file mode 100644 index 0000000000000..0efbed397abf9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 @@ -0,0 +1,4 @@ +sin(x) - returns the sine of x (x is in radians) +Example: + > SELECT sin(0) FROM src LIMIT 1; + 0 diff --git a/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab b/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 b/sql/hive/src/test/resources/golden/udf_sin-3-1d3a615e3aa252a317daa601811820b1 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 rename to sql/hive/src/test/resources/golden/udf_sin-3-1d3a615e3aa252a317daa601811820b1 diff --git a/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 b/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 deleted file mode 100644 index ee21925b0ccc5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 +++ /dev/null @@ -1 +0,0 @@ -0.8304973704919705 0.9999996829318346 -0.479425538604203 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a b/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a new file mode 100644 index 0000000000000..4ca4af756468c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a @@ -0,0 +1 @@ +0.8304973704919705 0.9999996829318346 -0.479425538604203 diff --git a/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 b/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 deleted file mode 100644 index d8952629349a9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 +++ /dev/null @@ -1 +0,0 @@ -size(a) - Returns the size of a diff --git a/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 b/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 deleted file mode 100644 index d8952629349a9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 +++ /dev/null @@ -1 +0,0 @@ -size(a) - Returns the size of a diff --git a/sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 b/sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd b/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd deleted file mode 100644 index 08a708b090282..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd +++ /dev/null @@ -1 +0,0 @@ -3 1 1 -1 diff --git a/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 b/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 index 572ecb0462eb7..132d6138e4983 100644 --- a/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 +++ b/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 @@ -1 +1 @@ -There is no documentation for function 'smallint' \ No newline at end of file +There is no documentation for function 'smallint' diff --git a/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 b/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 index 572ecb0462eb7..132d6138e4983 100644 --- a/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 +++ b/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 @@ -1 +1 @@ -There is no documentation for function 'smallint' \ No newline at end of file +There is no documentation for function 'smallint' diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 b/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 deleted file mode 100644 index d514df4191b89..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 +++ /dev/null @@ -1 +0,0 @@ -sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 b/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 deleted file mode 100644 index 43e36513de881..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 +++ /dev/null @@ -1,4 +0,0 @@ -sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. -Example: - > SELECT sort_array(array('b', 'd', 'c', 'a')) FROM src LIMIT 1; - 'a', 'b', 'c', 'd' diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de b/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 b/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 deleted file mode 100644 index 2e9458debfd0f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 +++ /dev/null @@ -1 +0,0 @@ -["a","b","c","d","e","f","g"] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e b/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e deleted file mode 100644 index 94f18d09863a7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e +++ /dev/null @@ -1 +0,0 @@ -["enterprise databases","hadoop distributed file system","hadoop map-reduce"] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 b/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 deleted file mode 100644 index e1968ef44a2eb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 +++ /dev/null @@ -1 +0,0 @@ -[1,2,3,4,5,6,7,8,9] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 b/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 deleted file mode 100644 index b6c12ace4162b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 +++ /dev/null @@ -1 +0,0 @@ -[-3.445,0.777,1.0,1.325,2.003,2.333,9.0] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 b/sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a b/sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 b/sql/hive/src/test/resources/golden/udf_space-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 rename to sql/hive/src/test/resources/golden/udf_space-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 b/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 deleted file mode 100644 index ed5bda787df23..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 +++ /dev/null @@ -1 +0,0 @@ -space(n) - returns n spaces \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 b/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 new file mode 100644 index 0000000000000..a443bc3cbf0bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 @@ -0,0 +1 @@ +space(n) - returns n spaces diff --git a/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 b/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 deleted file mode 100644 index 5713d4b0464be..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 +++ /dev/null @@ -1,4 +0,0 @@ -space(n) - returns n spaces -Example: - > SELECT space(2) FROM src LIMIT 1; - ' ' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 b/sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 b/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 new file mode 100644 index 0000000000000..25a7583ef01e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 @@ -0,0 +1,4 @@ +space(n) - returns n spaces +Example: + > SELECT space(2) FROM src LIMIT 1; + ' ' diff --git a/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 b/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 deleted file mode 100644 index 85a16b2abe5ef..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 +++ /dev/null @@ -1 +0,0 @@ -10 0 1 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 b/sql/hive/src/test/resources/golden/udf_space-3-a1b9dad63547f7ba73a5230d650983b0 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 rename to sql/hive/src/test/resources/golden/udf_space-3-a1b9dad63547f7ba73a5230d650983b0 diff --git a/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 b/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 deleted file mode 100644 index 8dfaf2745f666..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 +++ /dev/null @@ -1 +0,0 @@ - \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 b/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 new file mode 100644 index 0000000000000..6f07be9b1d043 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 @@ -0,0 +1 @@ +10 0 1 0 0 diff --git a/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 b/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 new file mode 100644 index 0000000000000..8f243e851f12f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 @@ -0,0 +1 @@ + diff --git a/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 b/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 deleted file mode 100644 index 6b183ccfb17b4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 +++ /dev/null @@ -1 +0,0 @@ -split(str, regex) - Splits str around occurances that match regex \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 b/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 deleted file mode 100644 index 9e5c522da155b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 +++ /dev/null @@ -1,4 +0,0 @@ -split(str, regex) - Splits str around occurances that match regex -Example: - > SELECT split('oneAtwoBthreeC', '[ABC]') FROM src LIMIT 1; - ["one", "two", "three"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 b/sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 b/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 deleted file mode 100644 index 9174f1a92557e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 +++ /dev/null @@ -1 +0,0 @@ -["a","b","c"] ["one","two","three"] [] ["5","4","1","2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 b/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 index eadd6eeb8b149..01efc31b30ede 100644 --- a/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 +++ b/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 @@ -1 +1 @@ -sqrt(x) - returns the square root of x \ No newline at end of file +sqrt(x) - returns the square root of x diff --git a/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 b/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 index 74df32f221278..16b77c4130fb6 100644 --- a/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 +++ b/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 @@ -1,4 +1,4 @@ sqrt(x) - returns the square root of x Example: > SELECT sqrt(4) FROM src LIMIT 1; - 2 \ No newline at end of file + 2 diff --git a/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b b/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b index 2966c01c1b7b2..5cedcfd415c88 100644 --- a/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b +++ b/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b @@ -1 +1 @@ -std(x) - Returns the standard deviation of a set of numbers \ No newline at end of file +std(x) - Returns the standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 b/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 index 388eaa1c7439a..d54ebfbd6fb1a 100644 --- a/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 +++ b/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 @@ -1,2 +1,2 @@ std(x) - Returns the standard deviation of a set of numbers -Synonyms: stddev_pop, stddev \ No newline at end of file +Synonyms: stddev_pop, stddev diff --git a/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 b/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 index 74c6e1eacc379..c8b11307792b3 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 +++ b/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 @@ -1 +1 @@ -stddev(x) - Returns the standard deviation of a set of numbers \ No newline at end of file +stddev(x) - Returns the standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d b/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d index 1c461b68c1440..5f674788180e8 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d +++ b/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d @@ -1,2 +1,2 @@ stddev(x) - Returns the standard deviation of a set of numbers -Synonyms: stddev_pop, std \ No newline at end of file +Synonyms: stddev_pop, std diff --git a/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 b/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 index 741771707f280..98b461bd09708 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 +++ b/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 @@ -1 +1 @@ -Function 'udf_stddev_pop' does not exist. \ No newline at end of file +Function 'udf_stddev_pop' does not exist. diff --git a/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 b/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 index 741771707f280..98b461bd09708 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 +++ b/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 @@ -1 +1 @@ -Function 'udf_stddev_pop' does not exist. \ No newline at end of file +Function 'udf_stddev_pop' does not exist. diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 index c1fb091d2c425..cc3c37142270e 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 +++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 @@ -1 +1 @@ -stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file +stddev_samp(x) - Returns the sample standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 index c1fb091d2c425..cc3c37142270e 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 +++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 @@ -1 +1 @@ -stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file +stddev_samp(x) - Returns the sample standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 index c1fb091d2c425..cc3c37142270e 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 +++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 @@ -1 +1 @@ -stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file +stddev_samp(x) - Returns the sample standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 index c1fb091d2c425..cc3c37142270e 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 +++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 @@ -1 +1 @@ -stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file +stddev_samp(x) - Returns the sample standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 b/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 index c61c4e1135410..4568e35cb1a21 100644 --- a/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 +++ b/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 @@ -1 +1 @@ -There is no documentation for function 'string' \ No newline at end of file +There is no documentation for function 'string' diff --git a/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d b/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d index c61c4e1135410..4568e35cb1a21 100644 --- a/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d +++ b/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d @@ -1 +1 @@ -There is no documentation for function 'string' \ No newline at end of file +There is no documentation for function 'string' diff --git a/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd b/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd deleted file mode 100644 index c4efdaebc8aab..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd +++ /dev/null @@ -1 +0,0 @@ -substr(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstr(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len diff --git a/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 b/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 deleted file mode 100644 index 44958996269c9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 +++ /dev/null @@ -1,10 +0,0 @@ -substr(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstr(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len -Synonyms: substring -pos is a 1-based index. If pos<0 the starting position is determined by counting backwards from the end of str. -Example: - > SELECT substr('Facebook', 5) FROM src LIMIT 1; - 'book' - > SELECT substr('Facebook', -5) FROM src LIMIT 1; - 'ebook' - > SELECT substr('Facebook', 5, 1) FROM src LIMIT 1; - 'b' diff --git a/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 b/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 deleted file mode 100644 index 9516b839a3956..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 b/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 deleted file mode 100644 index 7f6c057b5eebd..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 +++ /dev/null @@ -1 +0,0 @@ - \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a b/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a deleted file mode 100644 index 042ca15005869..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a +++ /dev/null @@ -1 +0,0 @@ -CDEF CDEF CDEFG CDEFG ABC ABC BC C ABC BC A A A \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 b/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 deleted file mode 100644 index 85346dc923b4a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 +++ /dev/null @@ -1 +0,0 @@ -A AB ABC ABC A AB ABC ABC B BC BC BC C C C C \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 b/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 deleted file mode 100644 index 9d607f4d84214..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 +++ /dev/null @@ -1 +0,0 @@ -C C C C B BC BC BC A AB ABC ABC \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 b/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 deleted file mode 100644 index 042ca15005869..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 +++ /dev/null @@ -1 +0,0 @@ -CDEF CDEF CDEFG CDEFG ABC ABC BC C ABC BC A A A \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade b/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade deleted file mode 100644 index c14d8d4279c5f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL NULL NULL A AB ABC ABC A AB ABC ABC B BC BC BC C C C C C C C C B BC BC BC A AB ABC ABC diff --git a/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d b/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d deleted file mode 100644 index 94a57d86c88fd..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d +++ /dev/null @@ -1 +0,0 @@ -玩 玩 玩玩玩 abc 玩玩玩 diff --git a/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece b/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece index b0ade52e69a9d..6a2376d85e7bc 100644 --- a/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece +++ b/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece @@ -1 +1 @@ -substring(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstring(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len \ No newline at end of file +substring(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstring(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len diff --git a/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 b/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 index 1d61dc4518087..32941e727ce38 100644 --- a/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 +++ b/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 @@ -7,4 +7,4 @@ Example: > SELECT substring('Facebook', -5) FROM src LIMIT 1; 'ebook' > SELECT substring('Facebook', 5, 1) FROM src LIMIT 1; - 'b' \ No newline at end of file + 'b' diff --git a/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 b/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 index 0f4a4dce9dd3d..1ad8be1242f00 100644 --- a/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 +++ b/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 @@ -1 +1 @@ -a - b - Returns the difference a-b \ No newline at end of file +a - b - Returns the difference a-b diff --git a/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 index 0f4a4dce9dd3d..1ad8be1242f00 100644 --- a/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 +++ b/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 @@ -1 +1 @@ -a - b - Returns the difference a-b \ No newline at end of file +a - b - Returns the difference a-b diff --git a/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 b/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 index 744935026110c..3c8fbd8c4d900 100644 --- a/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 +++ b/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 @@ -1 +1 @@ -sum(x) - Returns the sum of a set of numbers \ No newline at end of file +sum(x) - Returns the sum of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b b/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b index 744935026110c..3c8fbd8c4d900 100644 --- a/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b +++ b/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b @@ -1 +1 @@ -sum(x) - Returns the sum of a set of numbers \ No newline at end of file +sum(x) - Returns the sum of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 b/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 index 744935026110c..3c8fbd8c4d900 100644 --- a/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 +++ b/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 @@ -1 +1 @@ -sum(x) - Returns the sum of a set of numbers \ No newline at end of file +sum(x) - Returns the sum of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b b/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b index 744935026110c..3c8fbd8c4d900 100644 --- a/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b +++ b/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b @@ -1 +1 @@ -sum(x) - Returns the sum of a set of numbers \ No newline at end of file +sum(x) - Returns the sum of a set of numbers diff --git a/sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 b/sql/hive/src/test/resources/golden/udf_tan-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 rename to sql/hive/src/test/resources/golden/udf_tan-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 deleted file mode 100644 index c91072e69740b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 +++ /dev/null @@ -1 +0,0 @@ -tan(x) - returns the tangent of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 deleted file mode 100644 index 4cc0d466f00f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 +++ /dev/null @@ -1,4 +0,0 @@ -tan(x) - returns the tangent of x (x is in radians) -Example: - > SELECT tan(0) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 new file mode 100644 index 0000000000000..3ac5f19b2f2f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 @@ -0,0 +1 @@ +tan(x) - returns the tangent of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 b/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 new file mode 100644 index 0000000000000..f26ecfe4a29d3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 @@ -0,0 +1,4 @@ +tan(x) - returns the tangent of x (x is in radians) +Example: + > SELECT tan(0) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 b/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 deleted file mode 100644 index 04ec33c11c3b4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 +++ /dev/null @@ -1 +0,0 @@ -1.5574077246549023 -0.29100619138474915 -1.5574077246549023 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 b/sql/hive/src/test/resources/golden/udf_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 rename to sql/hive/src/test/resources/golden/udf_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b diff --git a/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c b/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c new file mode 100644 index 0000000000000..e9f2fa411fa33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c @@ -0,0 +1 @@ +1.5574077246549023 -0.29100619138474915 -1.5574077246549023 diff --git a/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 deleted file mode 100644 index c91072e69740b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 +++ /dev/null @@ -1 +0,0 @@ -tan(x) - returns the tangent of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 deleted file mode 100644 index 4cc0d466f00f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 +++ /dev/null @@ -1,4 +0,0 @@ -tan(x) - returns the tangent of x (x is in radians) -Example: - > SELECT tan(0) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 new file mode 100644 index 0000000000000..3ac5f19b2f2f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 @@ -0,0 +1 @@ +tan(x) - returns the tangent of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 b/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 new file mode 100644 index 0000000000000..f26ecfe4a29d3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 @@ -0,0 +1,4 @@ +tan(x) - returns the tangent of x (x is in radians) +Example: + > SELECT tan(0) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 b/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 deleted file mode 100644 index 04ec33c11c3b4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 +++ /dev/null @@ -1 +0,0 @@ -1.5574077246549023 -0.29100619138474915 -1.5574077246549023 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 b/sql/hive/src/test/resources/golden/udf_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 rename to sql/hive/src/test/resources/golden/udf_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b diff --git a/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c b/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c new file mode 100644 index 0000000000000..e9f2fa411fa33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c @@ -0,0 +1 @@ +1.5574077246549023 -0.29100619138474915 -1.5574077246549023 diff --git a/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad b/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad deleted file mode 100644 index 89617d8af0085..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION testlength 'org.apache.hadoop.hive.ql.udf.UDFTestLength') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 b/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 deleted file mode 100644 index 6d967660b3d41..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION testlength2 'org.apache.hadoop.hive.ql.udf.UDFTestLength2') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d b/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d index 1b13d42908539..898364b6d3e0b 100644 --- a/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d +++ b/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d @@ -1 +1 @@ -There is no documentation for function 'tinyint' \ No newline at end of file +There is no documentation for function 'tinyint' diff --git a/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 b/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 index 1b13d42908539..898364b6d3e0b 100644 --- a/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 +++ b/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 @@ -1 +1 @@ -There is no documentation for function 'tinyint' \ No newline at end of file +There is no documentation for function 'tinyint' diff --git a/sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 b/sql/hive/src/test/resources/golden/udf_to_byte-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 rename to sql/hive/src/test/resources/golden/udf_to_byte-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 b/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 b/sql/hive/src/test/resources/golden/udf_to_byte-1-94eb069fb446b7758f7e06386486bec9 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 rename to sql/hive/src/test/resources/golden/udf_to_byte-1-94eb069fb446b7758f7e06386486bec9 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 b/sql/hive/src/test/resources/golden/udf_to_byte-2-233102b562824cf38010868478e91e1 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 rename to sql/hive/src/test/resources/golden/udf_to_byte-2-233102b562824cf38010868478e91e1 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc b/sql/hive/src/test/resources/golden/udf_to_byte-3-5dc0e4c21764683d98700860d2c8ab31 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc rename to sql/hive/src/test/resources/golden/udf_to_byte-3-5dc0e4c21764683d98700860d2c8ab31 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 b/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b b/sql/hive/src/test/resources/golden/udf_to_byte-4-dafb27507b4d30fd2231680f9ea80c82 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b rename to sql/hive/src/test/resources/golden/udf_to_byte-4-dafb27507b4d30fd2231680f9ea80c82 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/udf_to_byte-5-eb6600cd2260e8e75253e7844c0d7dc2 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to sql/hive/src/test/resources/golden/udf_to_byte-5-eb6600cd2260e8e75253e7844c0d7dc2 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 b/sql/hive/src/test/resources/golden/udf_to_byte-6-489cd2d26b9efde2cdbff19254289371 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 rename to sql/hive/src/test/resources/golden/udf_to_byte-6-489cd2d26b9efde2cdbff19254289371 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 b/sql/hive/src/test/resources/golden/udf_to_byte-7-1eaba393f93af1763dd761172fb78d52 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 rename to sql/hive/src/test/resources/golden/udf_to_byte-7-1eaba393f93af1763dd761172fb78d52 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 b/sql/hive/src/test/resources/golden/udf_to_byte-8-8fe36cf8fba87514744a89fe50414f79 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 rename to sql/hive/src/test/resources/golden/udf_to_byte-8-8fe36cf8fba87514744a89fe50414f79 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 b/sql/hive/src/test/resources/golden/udf_to_byte-9-322163c32973ccc3a5168463db7a8589 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 rename to sql/hive/src/test/resources/golden/udf_to_byte-9-322163c32973ccc3a5168463db7a8589 diff --git a/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 b/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 index 54a99d2a94ef4..caf10b953f895 100644 --- a/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 +++ b/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 @@ -1 +1 @@ -to_date(expr) - Extracts the date part of the date or datetime expression expr \ No newline at end of file +to_date(expr) - Extracts the date part of the date or datetime expression expr diff --git a/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e b/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e index 81ddc59fcb531..d9cc4956e5f84 100644 --- a/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e +++ b/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e @@ -1,4 +1,4 @@ to_date(expr) - Extracts the date part of the date or datetime expression expr Example: - > SELECT to_date('2009-30-07 04:17:52') FROM src LIMIT 1; - '2009-30-07' \ No newline at end of file + > SELECT to_date('2009-07-30 04:17:52') FROM src LIMIT 1; + '2009-07-30' diff --git a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe deleted file mode 100644 index 319fde05380bc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe +++ /dev/null @@ -1 +0,0 @@ --7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f deleted file mode 100644 index 8c1c4fe62b6c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f +++ /dev/null @@ -1 +0,0 @@ --18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 deleted file mode 100644 index 1b650de78904f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 +++ /dev/null @@ -1 +0,0 @@ --129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 deleted file mode 100644 index 3a3bd0df03b5b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 +++ /dev/null @@ -1 +0,0 @@ --1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 deleted file mode 100644 index 38f7ad5afa0ab..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 +++ /dev/null @@ -1 +0,0 @@ --3.140000104904175 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 deleted file mode 100644 index f45d1f04dc920..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 +++ /dev/null @@ -1 +0,0 @@ --38.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 deleted file mode 100644 index 319fde05380bc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 +++ /dev/null @@ -1 +0,0 @@ --7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce deleted file mode 100644 index 8c1c4fe62b6c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce +++ /dev/null @@ -1 +0,0 @@ --18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 deleted file mode 100644 index 1b650de78904f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 +++ /dev/null @@ -1 +0,0 @@ --129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 deleted file mode 100644 index 3a3bd0df03b5b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 +++ /dev/null @@ -1 +0,0 @@ --1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c deleted file mode 100644 index f45d1f04dc920..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c +++ /dev/null @@ -1 +0,0 @@ --38.14 diff --git a/sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb b/sql/hive/src/test/resources/golden/udf_to_long-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb rename to sql/hive/src/test/resources/golden/udf_to_long-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 b/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 b/sql/hive/src/test/resources/golden/udf_to_long-1-8aeb3ba62d4a0cecfff363741e8042f6 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 rename to sql/hive/src/test/resources/golden/udf_to_long-1-8aeb3ba62d4a0cecfff363741e8042f6 diff --git a/sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 b/sql/hive/src/test/resources/golden/udf_to_long-2-7e8fa1ae8d00a121ec14941a48d24947 similarity index 100% rename from sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 rename to sql/hive/src/test/resources/golden/udf_to_long-2-7e8fa1ae8d00a121ec14941a48d24947 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 b/sql/hive/src/test/resources/golden/udf_to_long-3-6e5936fba8e7486beb9ab998548bbe9b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 rename to sql/hive/src/test/resources/golden/udf_to_long-3-6e5936fba8e7486beb9ab998548bbe9b diff --git a/sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 b/sql/hive/src/test/resources/golden/udf_to_long-4-8c284b082a256abf0426d4f6f1971703 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 rename to sql/hive/src/test/resources/golden/udf_to_long-4-8c284b082a256abf0426d4f6f1971703 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 b/sql/hive/src/test/resources/golden/udf_to_long-5-6bb29b93f6b0f1427ba93efb4e78810a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 rename to sql/hive/src/test/resources/golden/udf_to_long-5-6bb29b93f6b0f1427ba93efb4e78810a diff --git a/sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d b/sql/hive/src/test/resources/golden/udf_to_long-6-290b5a4ce01563482e81b3b532ebf9db similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d rename to sql/hive/src/test/resources/golden/udf_to_long-6-290b5a4ce01563482e81b3b532ebf9db diff --git a/sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 b/sql/hive/src/test/resources/golden/udf_to_long-7-da20f84586dac3e50ee9d5b9078f44db similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 rename to sql/hive/src/test/resources/golden/udf_to_long-7-da20f84586dac3e50ee9d5b9078f44db diff --git a/sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e b/sql/hive/src/test/resources/golden/udf_to_long-8-90f068b4b6275bdd1c4c431fb7fa90e2 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e rename to sql/hive/src/test/resources/golden/udf_to_long-8-90f068b4b6275bdd1c4c431fb7fa90e2 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 b/sql/hive/src/test/resources/golden/udf_to_long-9-cc8b79539085fe0e00f672b562c51cd0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 rename to sql/hive/src/test/resources/golden/udf_to_long-9-cc8b79539085fe0e00f672b562c51cd0 diff --git a/sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_to_short-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_to_short-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a b/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a deleted file mode 100644 index 7951defec192a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a +++ /dev/null @@ -1 +0,0 @@ -NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b b/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a b/sql/hive/src/test/resources/golden/udf_to_short-1-94f15fe043839493107058a06a210cf7 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a rename to sql/hive/src/test/resources/golden/udf_to_short-1-94f15fe043839493107058a06a210cf7 diff --git a/sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 b/sql/hive/src/test/resources/golden/udf_to_short-2-981e4cb6654fde7eb4634c7ad72f8570 similarity index 100% rename from sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 rename to sql/hive/src/test/resources/golden/udf_to_short-2-981e4cb6654fde7eb4634c7ad72f8570 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 b/sql/hive/src/test/resources/golden/udf_to_short-3-b259ee30ecf279bb4ad12d1515ca2767 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 rename to sql/hive/src/test/resources/golden/udf_to_short-3-b259ee30ecf279bb4ad12d1515ca2767 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce b/sql/hive/src/test/resources/golden/udf_to_short-4-40ffb132d5641645e2b8043dc056fb0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce rename to sql/hive/src/test/resources/golden/udf_to_short-4-40ffb132d5641645e2b8043dc056fb0 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 b/sql/hive/src/test/resources/golden/udf_to_short-5-5d6c46b0154d1073c035a79dbf612479 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 rename to sql/hive/src/test/resources/golden/udf_to_short-5-5d6c46b0154d1073c035a79dbf612479 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 b/sql/hive/src/test/resources/golden/udf_to_short-6-6561b41835a21f973cbbc2dd80eef87f similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 rename to sql/hive/src/test/resources/golden/udf_to_short-6-6561b41835a21f973cbbc2dd80eef87f diff --git a/sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 b/sql/hive/src/test/resources/golden/udf_to_short-7-9f83813005b639a23901ca6ff87ff473 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 rename to sql/hive/src/test/resources/golden/udf_to_short-7-9f83813005b639a23901ca6ff87ff473 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d b/sql/hive/src/test/resources/golden/udf_to_short-8-885656e165feb3a674cf636dbf08716c similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d rename to sql/hive/src/test/resources/golden/udf_to_short-8-885656e165feb3a674cf636dbf08716c diff --git a/sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 b/sql/hive/src/test/resources/golden/udf_to_short-9-750382fa1a1b3ed5dca0d549d3a68996 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 rename to sql/hive/src/test/resources/golden/udf_to_short-9-750382fa1a1b3ed5dca0d549d3a68996 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc b/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc deleted file mode 100644 index 7951defec192a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc +++ /dev/null @@ -1 +0,0 @@ -NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 b/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 deleted file mode 100644 index ef2f5130b8575..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 +++ /dev/null @@ -1 +0,0 @@ -TRUE diff --git a/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 b/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 b/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 deleted file mode 100644 index 6cc8a61f8f6ad..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 +++ /dev/null @@ -1 +0,0 @@ --18 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 b/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 deleted file mode 100644 index 9828ff22b667b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 +++ /dev/null @@ -1 +0,0 @@ --129 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 b/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 deleted file mode 100644 index 450a6125550e5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 +++ /dev/null @@ -1 +0,0 @@ --1025 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 b/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 b/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea b/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b b/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b deleted file mode 100644 index bc56c4d89448a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b +++ /dev/null @@ -1 +0,0 @@ -Foo diff --git a/sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/udf_translate-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 rename to sql/hive/src/test/resources/golden/udf_translate-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 b/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 deleted file mode 100644 index 4255dc76f501e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 +++ /dev/null @@ -1 +0,0 @@ -translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 b/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 new file mode 100644 index 0000000000000..e92c71fd3ccf9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 @@ -0,0 +1 @@ +translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string diff --git a/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 deleted file mode 100644 index e7beead53b399..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 +++ /dev/null @@ -1,13 +0,0 @@ -translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string -translate(string input, string from, string to) is an equivalent function to translate in PostGreSQL. It works on a character by character basis on the input string (first parameter). A character in the input is checked for presence in the from string (second parameter). If a match happens, the character from to string (third parameter) which appears at the same index as the character in from string is obtained. This character is emitted in the output string instead of the original character from the input string. If the to string is shorter than the from string, there may not be a character present at the same index in the to string. In such a case, nothing is emitted for the original character and it's deleted from the output string. -For example, - -translate('abcdef', 'adc', '19') returns '1b9ef' replacing 'a' with '1', 'd' with '9' and removing 'c' from the input string - -translate('a b c d', ' ', '') return 'abcd' removing all spaces from the input string - -If the same character is present multiple times in the input string, the first occurence of the character is the one that's considered for matching. However, it is not recommended to have the same character more than once in the from string since it's not required and adds to confusion. - -For example, - -translate('abcdef', 'ada', '192') returns '1bc9ef' replaces 'a' with '1' and 'd' with '9' ignoring the second occurence of 'a' in the from string mapping it to '2' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 b/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 deleted file mode 100644 index 7d44692e4f7c0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 +++ /dev/null @@ -1 +0,0 @@ -12cd 12cd \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae b/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae new file mode 100644 index 0000000000000..ae8343d33bc11 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae @@ -0,0 +1 @@ +NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a b/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a new file mode 100644 index 0000000000000..2e2f35d06c042 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a @@ -0,0 +1 @@ +12cd 12cd diff --git a/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b b/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b deleted file mode 100644 index 90e0d872f8bfc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b +++ /dev/null @@ -1 +0,0 @@ -123d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 b/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 deleted file mode 100644 index 0770d02cb2303..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 +++ /dev/null @@ -1 +0,0 @@ -Ãbcd \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf b/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf new file mode 100644 index 0000000000000..36284d8236d0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf @@ -0,0 +1 @@ +123d diff --git a/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 b/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 new file mode 100644 index 0000000000000..0208f1f4980c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 @@ -0,0 +1 @@ +Ãbcd diff --git a/sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 b/sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 new file mode 100644 index 0000000000000..9ced4ee32cf0b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 @@ -0,0 +1,13 @@ +translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string +translate(string input, string from, string to) is an equivalent function to translate in PostGreSQL. It works on a character by character basis on the input string (first parameter). A character in the input is checked for presence in the from string (second parameter). If a match happens, the character from to string (third parameter) which appears at the same index as the character in from string is obtained. This character is emitted in the output string instead of the original character from the input string. If the to string is shorter than the from string, there may not be a character present at the same index in the to string. In such a case, nothing is emitted for the original character and it's deleted from the output string. +For example, + +translate('abcdef', 'adc', '19') returns '1b9ef' replacing 'a' with '1', 'd' with '9' and removing 'c' from the input string + +translate('a b c d', ' ', '') return 'abcd' removing all spaces from the input string + +If the same character is present multiple times in the input string, the first occurence of the character is the one that's considered for matching. However, it is not recommended to have the same character more than once in the from string since it's not required and adds to confusion. + +For example, + +translate('abcdef', 'ada', '192') returns '1bc9ef' replaces 'a' with '1' and 'd' with '9' ignoring the second occurence of 'a' in the from string mapping it to '2' diff --git a/sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a b/sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 b/sql/hive/src/test/resources/golden/udf_translate-3-42aba80bf1913dd7c64545831f476c58 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 rename to sql/hive/src/test/resources/golden/udf_translate-3-42aba80bf1913dd7c64545831f476c58 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa b/sql/hive/src/test/resources/golden/udf_translate-4-20904c8be8fed5cbd2d66ead6248a60a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa rename to sql/hive/src/test/resources/golden/udf_translate-4-20904c8be8fed5cbd2d66ead6248a60a diff --git a/sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba b/sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 b/sql/hive/src/test/resources/golden/udf_translate-5-5d4abaf86254bacaa545c769bd7e50ba similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 rename to sql/hive/src/test/resources/golden/udf_translate-5-5d4abaf86254bacaa545c769bd7e50ba diff --git a/sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 b/sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b b/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b deleted file mode 100644 index 0a54ee40ecb13..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b +++ /dev/null @@ -1 +0,0 @@ -12cd 12d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/udf_translate-6-f2637240d227f9732d3db76f2e9d3a59 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad rename to sql/hive/src/test/resources/golden/udf_translate-6-f2637240d227f9732d3db76f2e9d3a59 diff --git a/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 b/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 deleted file mode 100644 index 0a54ee40ecb13..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 +++ /dev/null @@ -1 +0,0 @@ -12cd 12d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 b/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 new file mode 100644 index 0000000000000..fbf71ff5ad9eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 @@ -0,0 +1 @@ +12cd 12d diff --git a/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 b/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 new file mode 100644 index 0000000000000..fbf71ff5ad9eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 @@ -0,0 +1 @@ +12cd 12d diff --git a/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 b/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 deleted file mode 100644 index 50c123df9d1d3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 +++ /dev/null @@ -1 +0,0 @@ -1bc \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 b/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 deleted file mode 100644 index b98d7e1c34024..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 b/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 new file mode 100644 index 0000000000000..bb456baa0a608 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 @@ -0,0 +1 @@ +1bc diff --git a/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f b/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f index f0620a190ae4e..598aeb205875b 100644 --- a/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f +++ b/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f @@ -1 +1 @@ -trim(str) - Removes the leading and trailing space characters from str \ No newline at end of file +trim(str) - Removes the leading and trailing space characters from str diff --git a/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 b/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 index 49a13dd64b286..b3ec336a877ee 100644 --- a/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 +++ b/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 @@ -1,4 +1,4 @@ trim(str) - Removes the leading and trailing space characters from str Example: > SELECT trim(' facebook ') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d b/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d index 057c231dccf0e..66412c3ba77a4 100644 --- a/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d +++ b/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d @@ -1 +1 @@ -ucase(str) - Returns str with all characters changed to uppercase \ No newline at end of file +ucase(str) - Returns str with all characters changed to uppercase diff --git a/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f b/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f index 128d1e21fae83..814e4f0d39dcf 100644 --- a/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f +++ b/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f @@ -2,4 +2,4 @@ ucase(str) - Returns str with all characters changed to uppercase Synonyms: upper Example: > SELECT ucase('Facebook') FROM src LIMIT 1; - 'FACEBOOK' \ No newline at end of file + 'FACEBOOK' diff --git a/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 b/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 deleted file mode 100644 index 44b2a42cc26c5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 +++ /dev/null @@ -1 +0,0 @@ -unhex(str) - Converts hexadecimal argument to binary diff --git a/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 b/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 deleted file mode 100644 index 97af3b812a429..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 +++ /dev/null @@ -1,14 +0,0 @@ -unhex(str) - Converts hexadecimal argument to binary -Performs the inverse operation of HEX(str). That is, it interprets -each pair of hexadecimal digits in the argument as a number and -converts it to the byte representation of the number. The -resulting characters are returned as a binary string. - -Example: -> SELECT DECODE(UNHEX('4D7953514C'), 'UTF-8') from src limit 1; -'MySQL' - -The characters in the argument string must be legal hexadecimal -digits: '0' .. '9', 'A' .. 'F', 'a' .. 'f'. If UNHEX() encounters -any nonhexadecimal digits in the argument, it returns NULL. Also, -if there are an odd number of characters a leading 0 is appended. diff --git a/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 b/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 deleted file mode 100644 index b4a6f2b692227..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 +++ /dev/null @@ -1 +0,0 @@ -MySQL 1267 a -4 diff --git a/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 b/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 deleted file mode 100644 index 3a67adaf0a9a8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c b/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c deleted file mode 100644 index 3f3bb2b7ce02e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c +++ /dev/null @@ -1 +0,0 @@ -create_union(tag, obj1, obj2, obj3, ...) - Creates a union with the object for given tag \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 b/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 deleted file mode 100644 index 748ed97055401..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 +++ /dev/null @@ -1,4 +0,0 @@ -create_union(tag, obj1, obj2, obj3, ...) - Creates a union with the object for given tag -Example: - > SELECT create_union(1, 1, "one") FROM src LIMIT 1; - one \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 b/sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 b/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 deleted file mode 100644 index 4b9ae5402aa53..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 +++ /dev/null @@ -1,2 +0,0 @@ -{0:238} {1:"val_238"} {1:{"col1":2,"col2":"b"}} -{0:86} {0:2.0} {1:{"col1":2,"col2":"b"}} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb b/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-d555c8cd733572bfa8cd3362da9480cb similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-1-d555c8cd733572bfa8cd3362da9480cb diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-8a9dbadae706047715cf5f903ff4a724 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-2-8a9dbadae706047715cf5f903ff4a724 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-28c40e51e55bed62693e626efda5d9c5 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-3-28c40e51e55bed62693e626efda5d9c5 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed b/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-6059ff48788d0fb8317fd331172ecea9 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-4-6059ff48788d0fb8317fd331172ecea9 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-b2e42ebb75cecf09961d36587797f6d0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-5-b2e42ebb75cecf09961d36587797f6d0 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-31243f5cb64356425b9f95ba011ac9d6 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-6-31243f5cb64356425b9f95ba011ac9d6 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-9b0f20bde1aaf9102b67a5498b167f31 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-7-9b0f20bde1aaf9102b67a5498b167f31 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d b/sql/hive/src/test/resources/golden/udf_unix_timestamp-8-47f433ff6ccce4c666440cc1a228a96d similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-8-47f433ff6ccce4c666440cc1a228a96d diff --git a/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead b/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead index 2e239cca5b860..40014886d9646 100644 --- a/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead +++ b/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead @@ -1 +1 @@ -upper(str) - Returns str with all characters changed to uppercase \ No newline at end of file +upper(str) - Returns str with all characters changed to uppercase diff --git a/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d b/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d index 673307e284b2e..0fb1bf1823d84 100644 --- a/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d +++ b/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d @@ -2,4 +2,4 @@ upper(str) - Returns str with all characters changed to uppercase Synonyms: ucase Example: > SELECT upper('Facebook') FROM src LIMIT 1; - 'FACEBOOK' \ No newline at end of file + 'FACEBOOK' diff --git a/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de index 9dbf241e28107..79e293fd4ed52 100644 --- a/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de +++ b/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de @@ -1 +1 @@ -var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file +var_pop(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 index 234a65bd8255d..4b4de5db812e9 100644 --- a/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 +++ b/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 @@ -1,2 +1,2 @@ var_pop(x) - Returns the variance of a set of numbers -Synonyms: variance \ No newline at end of file +Synonyms: variance diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 b/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 index 673deeb031b44..42a442f7a2ded 100644 --- a/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 +++ b/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 @@ -1 +1 @@ -var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file +var_samp(x) - Returns the sample variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b b/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b index 673deeb031b44..42a442f7a2ded 100644 --- a/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b +++ b/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b @@ -1 +1 @@ -var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file +var_samp(x) - Returns the sample variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 b/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 index 673deeb031b44..42a442f7a2ded 100644 --- a/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 +++ b/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 @@ -1 +1 @@ -var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file +var_samp(x) - Returns the sample variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b b/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b index 673deeb031b44..42a442f7a2ded 100644 --- a/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b +++ b/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b @@ -1 +1 @@ -var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file +var_samp(x) - Returns the sample variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 b/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 index f58ae0769d5a9..260300b7d3bd9 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 +++ b/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 @@ -1 +1 @@ -variance(x) - Returns the variance of a set of numbers \ No newline at end of file +variance(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a b/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a index e17528e607aba..b14cf43bc0a95 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a +++ b/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a @@ -1,2 +1,2 @@ variance(x) - Returns the variance of a set of numbers -Synonyms: var_pop \ No newline at end of file +Synonyms: var_pop diff --git a/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de index 9dbf241e28107..79e293fd4ed52 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de +++ b/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de @@ -1 +1 @@ -var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file +var_pop(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 index 234a65bd8255d..4b4de5db812e9 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 +++ b/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 @@ -1,2 +1,2 @@ var_pop(x) - Returns the variance of a set of numbers -Synonyms: variance \ No newline at end of file +Synonyms: variance diff --git a/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 b/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 index f58ae0769d5a9..260300b7d3bd9 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 +++ b/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 @@ -1 +1 @@ -variance(x) - Returns the variance of a set of numbers \ No newline at end of file +variance(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a b/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a index e17528e607aba..b14cf43bc0a95 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a +++ b/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a @@ -1,2 +1,2 @@ variance(x) - Returns the variance of a set of numbers -Synonyms: var_pop \ No newline at end of file +Synonyms: var_pop diff --git a/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de index 9dbf241e28107..79e293fd4ed52 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de +++ b/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de @@ -1 +1 @@ -var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file +var_pop(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 index 234a65bd8255d..4b4de5db812e9 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 +++ b/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 @@ -1,2 +1,2 @@ var_pop(x) - Returns the variance of a set of numbers -Synonyms: variance \ No newline at end of file +Synonyms: variance diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 b/sql/hive/src/test/resources/golden/udf_weekofyear-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 rename to sql/hive/src/test/resources/golden/udf_weekofyear-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 b/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 deleted file mode 100644 index 29e3b370b03b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 +++ /dev/null @@ -1 +0,0 @@ -weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd b/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd deleted file mode 100644 index c7939a11937c5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd +++ /dev/null @@ -1,6 +0,0 @@ -weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days. -Examples: - > SELECT weekofyear('2008-02-20') FROM src LIMIT 1; - 8 - > SELECT weekofyear('1980-12-31 12:59:59') FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 b/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 new file mode 100644 index 0000000000000..dca04d210339a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 @@ -0,0 +1 @@ +weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days. diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 b/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 deleted file mode 100644 index 20d1dc50ce1f7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 +++ /dev/null @@ -1 +0,0 @@ -1 1 2 1 52 8 52 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd b/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd new file mode 100644 index 0000000000000..337257dabd8d1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd @@ -0,0 +1,6 @@ +weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days. +Examples: + > SELECT weekofyear('2008-02-20') FROM src LIMIT 1; + 8 + > SELECT weekofyear('1980-12-31 12:59:59') FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 b/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 new file mode 100644 index 0000000000000..9d0163569f7b3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 @@ -0,0 +1 @@ +1 1 2 1 52 8 52 1 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 b/sql/hive/src/test/resources/golden/udf_when-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 rename to sql/hive/src/test/resources/golden/udf_when-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 b/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 deleted file mode 100644 index 2e36162dbaa33..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 +++ /dev/null @@ -1 +0,0 @@ -There is no documentation for function 'when' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 b/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 deleted file mode 100644 index 2e36162dbaa33..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 +++ /dev/null @@ -1 +0,0 @@ -There is no documentation for function 'when' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 b/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 new file mode 100644 index 0000000000000..3626da8447ca0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 @@ -0,0 +1 @@ +There is no documentation for function 'when' diff --git a/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 b/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 new file mode 100644 index 0000000000000..3626da8447ca0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 @@ -0,0 +1 @@ +There is no documentation for function 'when' diff --git a/sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 b/sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 b/sql/hive/src/test/resources/golden/udf_when-3-734890c41528b9d918db66b0582228a4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 rename to sql/hive/src/test/resources/golden/udf_when-3-734890c41528b9d918db66b0582228a4 diff --git a/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f b/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f deleted file mode 100644 index 6929fdca28eb8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f +++ /dev/null @@ -1 +0,0 @@ -2 9 14 NULL 24 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 b/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 new file mode 100644 index 0000000000000..b28b0050514f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 @@ -0,0 +1 @@ +2 9 14 NULL 24 NULL diff --git a/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 b/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 new file mode 100644 index 0000000000000..db46b5a72aa80 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 @@ -0,0 +1 @@ +123 123.0 abcd diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 b/sql/hive/src/test/resources/golden/udf_xpath-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 rename to sql/hive/src/test/resources/golden/udf_xpath-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 b/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 deleted file mode 100644 index 18d9720653a5b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 +++ /dev/null @@ -1 +0,0 @@ -xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 b/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 deleted file mode 100644 index 3b092dafd32c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 +++ /dev/null @@ -1,8 +0,0 @@ -xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression -Example: - > SELECT xpath('b1b2b3c1c2', 'a/text()') FROM src LIMIT 1 - [] - > SELECT xpath('b1b2b3c1c2', 'a/b/text()') FROM src LIMIT 1 - ["b1","b2","b3"] - > SELECT xpath('b1b2b3c1c2', 'a/c/text()') FROM src LIMIT 1 - ["c1","c2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 b/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 new file mode 100644 index 0000000000000..d53b034f8fc6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 @@ -0,0 +1 @@ +xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 b/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 new file mode 100644 index 0000000000000..bff795649a7ea --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 @@ -0,0 +1,8 @@ +xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression +Example: + > SELECT xpath('b1b2b3c1c2', 'a/text()') FROM src LIMIT 1 + [] + > SELECT xpath('b1b2b3c1c2', 'a/b/text()') FROM src LIMIT 1 + ["b1","b2","b3"] + > SELECT xpath('b1b2b3c1c2', 'a/c/text()') FROM src LIMIT 1 + ["c1","c2"] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d b/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d deleted file mode 100644 index 0637a088a01e8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d +++ /dev/null @@ -1 +0,0 @@ -[] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea b/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea deleted file mode 100644 index 8e8aff2387621..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea +++ /dev/null @@ -1 +0,0 @@ -["b1","b2","b3","c1","c2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 b/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 new file mode 100644 index 0000000000000..fe51488c7066f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 @@ -0,0 +1 @@ +[] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a b/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a deleted file mode 100644 index e518b1f37e124..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a +++ /dev/null @@ -1 +0,0 @@ -["b1","b2","b3"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e b/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e new file mode 100644 index 0000000000000..96e245a7eb37c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e @@ -0,0 +1 @@ +["b1","b2","b3","c1","c2"] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 b/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 deleted file mode 100644 index 45615b772ba35..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 +++ /dev/null @@ -1 +0,0 @@ -["c1","c2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 b/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 new file mode 100644 index 0000000000000..44def14ea2c68 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 @@ -0,0 +1 @@ +["b1","b2","b3"] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f b/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f deleted file mode 100644 index 327bf3a1e76cc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f +++ /dev/null @@ -1 +0,0 @@ -["b1","c1"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a b/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a new file mode 100644 index 0000000000000..6a97cc328aa70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a @@ -0,0 +1 @@ +["c1","c2"] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 b/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 new file mode 100644 index 0000000000000..8b4abae5acff8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 @@ -0,0 +1 @@ +["b1","c1"] diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 deleted file mode 100644 index 134e9e99b7423..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 +++ /dev/null @@ -1 +0,0 @@ -xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 new file mode 100644 index 0000000000000..d47e3fe7d34a1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 @@ -0,0 +1 @@ +xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 deleted file mode 100644 index 624889cc0ac67..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 +++ /dev/null @@ -1,6 +0,0 @@ -xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression -Example: - > SELECT xpath_boolean('1','a/b') FROM src LIMIT 1; - true - > SELECT xpath_boolean('1','a/b = 2') FROM src LIMIT 1; - false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 new file mode 100644 index 0000000000000..4f3c2652aad11 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 @@ -0,0 +1,6 @@ +xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression +Example: + > SELECT xpath_boolean('1','a/b') FROM src LIMIT 1; + true + > SELECT xpath_boolean('1','a/b = 2') FROM src LIMIT 1; + false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-9b97a08303a7a89e7575687f6d7ba435 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-3-9b97a08303a7a89e7575687f6d7ba435 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 deleted file mode 100644 index 02e4a84d62c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 +++ /dev/null @@ -1 +0,0 @@ -false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df b/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-284ce7d6fc6850ca853111145784286b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-4-284ce7d6fc6850ca853111145784286b diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a b/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-95a0dfa897ba9050ad751a78aeb72f3d similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-5-95a0dfa897ba9050ad751a78aeb72f3d diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 deleted file mode 100644 index 02e4a84d62c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 +++ /dev/null @@ -1 +0,0 @@ -false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f deleted file mode 100644 index 02e4a84d62c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f +++ /dev/null @@ -1 +0,0 @@ -false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-796c717c9d52d3efa374d12fe65259e6 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-6-796c717c9d52d3efa374d12fe65259e6 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-8-62a1af222d4e12c59cef71b979c6e58 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-8-62a1af222d4e12c59cef71b979c6e58 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 b/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 deleted file mode 100644 index b56f4d00a8a74..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 +++ /dev/null @@ -1 +0,0 @@ -xpath_number(xml, xpath) - Returns a double value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_xpath_double-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_xpath_double-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 b/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 deleted file mode 100644 index b6b6227174647..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 +++ /dev/null @@ -1,5 +0,0 @@ -xpath_number(xml, xpath) - Returns a double value that matches the xpath expression -Synonyms: xpath_double -Example: - > SELECT xpath_number('12','sum(a/b)') FROM src LIMIT 1; - 3.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 b/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 new file mode 100644 index 0000000000000..a9cb2ecb60e28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 @@ -0,0 +1 @@ +xpath_number(xml, xpath) - Returns a double value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 b/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 new file mode 100644 index 0000000000000..fe6b09a7d14cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 @@ -0,0 +1 @@ +15.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 b/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 deleted file mode 100644 index 7104585f9a439..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 +++ /dev/null @@ -1 +0,0 @@ -7.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 b/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 deleted file mode 100644 index 6e6366051638f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 +++ /dev/null @@ -1 +0,0 @@ -5.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 b/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 new file mode 100644 index 0000000000000..4fedf1d20e157 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 @@ -0,0 +1 @@ +7.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb b/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb new file mode 100644 index 0000000000000..819e07a22435f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb @@ -0,0 +1 @@ +5.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 b/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 new file mode 100644 index 0000000000000..0f303917eb1bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 @@ -0,0 +1,5 @@ +xpath_number(xml, xpath) - Returns a double value that matches the xpath expression +Synonyms: xpath_double +Example: + > SELECT xpath_number('12','sum(a/b)') FROM src LIMIT 1; + 3.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 b/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 deleted file mode 100644 index 5b6de7db2f9f3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 +++ /dev/null @@ -1 +0,0 @@ -xpath_double(xml, xpath) - Returns a double value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 b/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 new file mode 100644 index 0000000000000..20ff7713a919d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 @@ -0,0 +1 @@ +xpath_double(xml, xpath) - Returns a double value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 b/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 deleted file mode 100644 index b8accbcae1e17..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 +++ /dev/null @@ -1,5 +0,0 @@ -xpath_double(xml, xpath) - Returns a double value that matches the xpath expression -Synonyms: xpath_number -Example: - > SELECT xpath_double('12','sum(a/b)') FROM src LIMIT 1; - 3.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c b/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 b/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 new file mode 100644 index 0000000000000..89c174f371b88 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 @@ -0,0 +1,5 @@ +xpath_double(xml, xpath) - Returns a double value that matches the xpath expression +Synonyms: xpath_number +Example: + > SELECT xpath_double('12','sum(a/b)') FROM src LIMIT 1; + 3.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 b/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 b/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 b/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 deleted file mode 100644 index 0a8ac8c629540..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 +++ /dev/null @@ -1 +0,0 @@ -8.0E19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 b/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c b/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 b/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 new file mode 100644 index 0000000000000..f65000a36a478 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 @@ -0,0 +1 @@ +8.0E19 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d b/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d deleted file mode 100644 index 9f8e9b69a33f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d +++ /dev/null @@ -1 +0,0 @@ -1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 b/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e b/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e deleted file mode 100644 index 3d3be3c32ed85..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e +++ /dev/null @@ -1 +0,0 @@ -15.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 b/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 b/sql/hive/src/test/resources/golden/udf_xpath_float-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 rename to sql/hive/src/test/resources/golden/udf_xpath_float-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 b/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 deleted file mode 100644 index ea4df0ae58e12..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 +++ /dev/null @@ -1 +0,0 @@ -xpath_float(xml, xpath) - Returns a float value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 b/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 new file mode 100644 index 0000000000000..b7dea42c78212 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 @@ -0,0 +1 @@ +xpath_float(xml, xpath) - Returns a float value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a b/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a deleted file mode 100644 index 6bc4d4a46de72..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a +++ /dev/null @@ -1,4 +0,0 @@ -xpath_float(xml, xpath) - Returns a float value that matches the xpath expression -Example: - > SELECT xpath_float('12','sum(a/b)') FROM src LIMIT 1; - 3.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab b/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab new file mode 100644 index 0000000000000..819e07a22435f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab @@ -0,0 +1 @@ +5.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 b/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a b/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a new file mode 100644 index 0000000000000..02b139e7a3610 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a @@ -0,0 +1,4 @@ +xpath_float(xml, xpath) - Returns a float value that matches the xpath expression +Example: + > SELECT xpath_float('12','sum(a/b)') FROM src LIMIT 1; + 3.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 b/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce b/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 deleted file mode 100644 index 0a8ac8c629540..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 +++ /dev/null @@ -1 +0,0 @@ -8.0E19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f b/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 b/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 new file mode 100644 index 0000000000000..f65000a36a478 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 @@ -0,0 +1 @@ +8.0E19 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 b/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf b/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf deleted file mode 100644 index 9f8e9b69a33f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf +++ /dev/null @@ -1 +0,0 @@ -1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 b/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa b/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa deleted file mode 100644 index 3d3be3c32ed85..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa +++ /dev/null @@ -1 +0,0 @@ -15.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 b/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 deleted file mode 100644 index 7104585f9a439..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 +++ /dev/null @@ -1 +0,0 @@ -7.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 b/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 new file mode 100644 index 0000000000000..fe6b09a7d14cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 @@ -0,0 +1 @@ +15.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 b/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 deleted file mode 100644 index 6e6366051638f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 +++ /dev/null @@ -1 +0,0 @@ -5.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a b/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a new file mode 100644 index 0000000000000..4fedf1d20e157 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a @@ -0,0 +1 @@ +7.0 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 b/sql/hive/src/test/resources/golden/udf_xpath_int-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 rename to sql/hive/src/test/resources/golden/udf_xpath_int-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 b/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 deleted file mode 100644 index 67ab19492e405..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 +++ /dev/null @@ -1 +0,0 @@ -xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 b/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 deleted file mode 100644 index 33349b0b22ad9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 +++ /dev/null @@ -1,4 +0,0 @@ -xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression -Example: - > SELECT xpath_int('12','sum(a/b)') FROM src LIMIT 1; - 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 b/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 new file mode 100644 index 0000000000000..000f90d69e7f7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 @@ -0,0 +1 @@ +xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a b/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 b/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 new file mode 100644 index 0000000000000..f969b9a48d10f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 @@ -0,0 +1,4 @@ +xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression +Example: + > SELECT xpath_int('12','sum(a/b)') FROM src LIMIT 1; + 3 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba b/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_xpath_int-3-a04ed6832ab542d6ee5903039511a826 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_xpath_int-3-a04ed6832ab542d6ee5903039511a826 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba b/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 b/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 deleted file mode 100644 index 55a89f5564d21..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 +++ /dev/null @@ -1 +0,0 @@ -2147483647 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_xpath_int-4-bf5a4dbb7a98abc91111a3798b56809f similarity index 100% rename from sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_xpath_int-4-bf5a4dbb7a98abc91111a3798b56809f diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 b/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 b/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 new file mode 100644 index 0000000000000..a51fa7d1efef6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 @@ -0,0 +1 @@ +2147483647 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a b/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_int-6-ac509f06f01c02924adef220404fc515 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf rename to sql/hive/src/test/resources/golden/udf_xpath_int-6-ac509f06f01c02924adef220404fc515 diff --git a/sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d b/sql/hive/src/test/resources/golden/udf_xpath_int-7-87ff12d650afb8f511d906778450fae7 similarity index 100% rename from sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d rename to sql/hive/src/test/resources/golden/udf_xpath_int-7-87ff12d650afb8f511d906778450fae7 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 b/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 deleted file mode 100644 index 3f10ffe7a4c47..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 +++ /dev/null @@ -1 +0,0 @@ -15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 b/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 b/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 deleted file mode 100644 index c7930257dfef5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 +++ /dev/null @@ -1 +0,0 @@ -7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 b/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 new file mode 100644 index 0000000000000..7f8f011eb73d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 @@ -0,0 +1 @@ +7 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 b/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 deleted file mode 100644 index 7813681f5b41c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 +++ /dev/null @@ -1 +0,0 @@ -5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_long-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf rename to sql/hive/src/test/resources/golden/udf_xpath_long-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b b/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b deleted file mode 100644 index 748fd827e8545..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b +++ /dev/null @@ -1 +0,0 @@ -xpath_long(xml, xpath) - Returns a long value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 b/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 deleted file mode 100644 index 65241b6ce9b64..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 +++ /dev/null @@ -1,4 +0,0 @@ -xpath_long(xml, xpath) - Returns a long value that matches the xpath expression -Example: - > SELECT xpath_long('12','sum(a/b)') FROM src LIMIT 1; - 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b b/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b new file mode 100644 index 0000000000000..ee1a589d6652b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b @@ -0,0 +1 @@ +xpath_long(xml, xpath) - Returns a long value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 b/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 b/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 new file mode 100644 index 0000000000000..b9eb5a4a99c02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 @@ -0,0 +1,4 @@ +xpath_long(xml, xpath) - Returns a long value that matches the xpath expression +Example: + > SELECT xpath_long('12','sum(a/b)') FROM src LIMIT 1; + 3 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b b/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_long-3-9df8d27e31d96e0b35b9b40910d4bd98 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf rename to sql/hive/src/test/resources/golden/udf_xpath_long-3-9df8d27e31d96e0b35b9b40910d4bd98 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d b/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/udf_xpath_long-4-3211913c56521887d30e3d1a50762b3f similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to sql/hive/src/test/resources/golden/udf_xpath_long-4-3211913c56521887d30e3d1a50762b3f diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 b/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 deleted file mode 100644 index 996d127e59365..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 +++ /dev/null @@ -1 +0,0 @@ -9223372036854775807 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d b/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 b/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 new file mode 100644 index 0000000000000..2045006edaf5e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 @@ -0,0 +1 @@ +9223372036854775807 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/udf_xpath_long-6-b695348ed3faec63be2c07d0d4afaaf3 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 rename to sql/hive/src/test/resources/golden/udf_xpath_long-6-b695348ed3faec63be2c07d0d4afaaf3 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 b/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 b/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 deleted file mode 100644 index 3f10ffe7a4c47..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 +++ /dev/null @@ -1 +0,0 @@ -15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc b/sql/hive/src/test/resources/golden/udf_xpath_long-7-ed5af6d7451107a753b2c7ff130ac73b similarity index 100% rename from sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc rename to sql/hive/src/test/resources/golden/udf_xpath_long-7-ed5af6d7451107a753b2c7ff130ac73b diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 b/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 b/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 deleted file mode 100644 index c7930257dfef5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 +++ /dev/null @@ -1 +0,0 @@ -7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 b/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 deleted file mode 100644 index 7813681f5b41c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 +++ /dev/null @@ -1 +0,0 @@ -5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd b/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd new file mode 100644 index 0000000000000..7f8f011eb73d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd @@ -0,0 +1 @@ +7 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/udf_xpath_short-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f rename to sql/hive/src/test/resources/golden/udf_xpath_short-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 b/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 deleted file mode 100644 index 22ccb9ca9defa..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 +++ /dev/null @@ -1 +0,0 @@ -xpath_short(xml, xpath) - Returns a short value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 b/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 deleted file mode 100644 index 931ce2abb3d4d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 +++ /dev/null @@ -1,4 +0,0 @@ -xpath_short(xml, xpath) - Returns a short value that matches the xpath expression -Example: - > SELECT xpath_short('12','sum(a/b)') FROM src LIMIT 1; - 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 b/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 new file mode 100644 index 0000000000000..63ab4f15e8c84 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 @@ -0,0 +1 @@ +xpath_short(xml, xpath) - Returns a short value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 b/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 b/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 b/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 new file mode 100644 index 0000000000000..ecb5e9a83109c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 @@ -0,0 +1,4 @@ +xpath_short(xml, xpath) - Returns a short value that matches the xpath expression +Example: + > SELECT xpath_short('12','sum(a/b)') FROM src LIMIT 1; + 3 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 b/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udf_xpath_short-3-c09b4ae6886fa58dcdd728bef45e7efa similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f rename to sql/hive/src/test/resources/golden/udf_xpath_short-3-c09b4ae6886fa58dcdd728bef45e7efa diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/udf_xpath_short-4-84f5f6bb2fdc0987d281d52a53a4b24e similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 rename to sql/hive/src/test/resources/golden/udf_xpath_short-4-84f5f6bb2fdc0987d281d52a53a4b24e diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c b/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c deleted file mode 100644 index d7d17fcbef95c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c +++ /dev/null @@ -1 +0,0 @@ --1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e b/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/udf_xpath_short-5-c09fd0565ed041c773fee9bd0436e861 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 rename to sql/hive/src/test/resources/golden/udf_xpath_short-5-c09fd0565ed041c773fee9bd0436e861 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/udf_xpath_short-6-16ced3de15d4ec87a4e7001376551758 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 rename to sql/hive/src/test/resources/golden/udf_xpath_short-6-16ced3de15d4ec87a4e7001376551758 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 b/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b b/sql/hive/src/test/resources/golden/udf_xpath_short-7-8ffdf20c15f3ed81bb5a92c61d200ae2 similarity index 100% rename from sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b rename to sql/hive/src/test/resources/golden/udf_xpath_short-7-8ffdf20c15f3ed81bb5a92c61d200ae2 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d b/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d deleted file mode 100644 index 3f10ffe7a4c47..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d +++ /dev/null @@ -1 +0,0 @@ -15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 b/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 b/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 deleted file mode 100644 index c7930257dfef5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 +++ /dev/null @@ -1 +0,0 @@ -7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e b/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e deleted file mode 100644 index 7813681f5b41c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e +++ /dev/null @@ -1 +0,0 @@ -5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d b/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d new file mode 100644 index 0000000000000..7f8f011eb73d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d @@ -0,0 +1 @@ +7 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_string-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf rename to sql/hive/src/test/resources/golden/udf_xpath_string-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 b/sql/hive/src/test/resources/golden/udf_xpath_string-1-e315d11e9feb29177b5cb1e221c7cfa4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 rename to sql/hive/src/test/resources/golden/udf_xpath_string-1-e315d11e9feb29177b5cb1e221c7cfa4 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc b/sql/hive/src/test/resources/golden/udf_xpath_string-10-d87fb71039c9d2419d750a0721c5696f similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc rename to sql/hive/src/test/resources/golden/udf_xpath_string-10-d87fb71039c9d2419d750a0721c5696f diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 b/sql/hive/src/test/resources/golden/udf_xpath_string-2-110b583cde6cd23c486d8223c444cbe9 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 rename to sql/hive/src/test/resources/golden/udf_xpath_string-2-110b583cde6cd23c486d8223c444cbe9 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f b/sql/hive/src/test/resources/golden/udf_xpath_string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f rename to sql/hive/src/test/resources/golden/udf_xpath_string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a b/sql/hive/src/test/resources/golden/udf_xpath_string-4-302630fe7dac2cc61fe7d36ead0f41ab similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a rename to sql/hive/src/test/resources/golden/udf_xpath_string-4-302630fe7dac2cc61fe7d36ead0f41ab diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a b/sql/hive/src/test/resources/golden/udf_xpath_string-5-19357ba9cb87d3a5717543d2afdc96e2 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a rename to sql/hive/src/test/resources/golden/udf_xpath_string-5-19357ba9cb87d3a5717543d2afdc96e2 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 b/sql/hive/src/test/resources/golden/udf_xpath_string-6-4837e470f745487fae4f498b3a2946bc similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 rename to sql/hive/src/test/resources/golden/udf_xpath_string-6-4837e470f745487fae4f498b3a2946bc diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 b/sql/hive/src/test/resources/golden/udf_xpath_string-7-8e42951d002e3c4034b4a51928442706 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 rename to sql/hive/src/test/resources/golden/udf_xpath_string-7-8e42951d002e3c4034b4a51928442706 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f b/sql/hive/src/test/resources/golden/udf_xpath_string-8-fdfa4e17d70608dcc634c9e1e8a8f288 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f rename to sql/hive/src/test/resources/golden/udf_xpath_string-8-fdfa4e17d70608dcc634c9e1e8a8f288 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 b/sql/hive/src/test/resources/golden/udf_xpath_string-9-84109613320bd05abccd1058044d62c3 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 rename to sql/hive/src/test/resources/golden/udf_xpath_string-9-84109613320bd05abccd1058044d62c3 diff --git a/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a b/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a deleted file mode 100644 index 599bf1880a83a..0000000000000 --- a/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a +++ /dev/null @@ -1 +0,0 @@ -stack(n, cols...) - turns k columns into n rows of size k/n each diff --git a/sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 b/sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 b/sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 index e82f93fe65c5e..f00ecaf75c084 100644 --- a/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 +++ b/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 @@ -1,18 +1,18 @@ # col_name data_type comment -a string None +a string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 18:39:52 PST 2014 +CreateTime: Tue Oct 21 05:37:51 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388803192 + transient_lastDdlTime 1413895071 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Bucket Columns: [] Sort Columns: [] Storage Desc Params: field.delim \u0001 - serialization.format \u0001 \ No newline at end of file + serialization.format \u0001 diff --git a/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 index e82f93fe65c5e..f00ecaf75c084 100644 --- a/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 +++ b/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 @@ -1,18 +1,18 @@ # col_name data_type comment -a string None +a string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 18:39:52 PST 2014 +CreateTime: Tue Oct 21 05:37:51 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388803192 + transient_lastDdlTime 1413895071 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Bucket Columns: [] Sort Columns: [] Storage Desc Params: field.delim \u0001 - serialization.format \u0001 \ No newline at end of file + serialization.format \u0001 diff --git a/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 index 2be72c6e4e8c7..79e197569a51c 100644 --- a/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 +++ b/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 @@ -1,18 +1,18 @@ # col_name data_type comment -a string None +a string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 18:39:52 PST 2014 +CreateTime: Tue Oct 21 05:37:51 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388803192 + transient_lastDdlTime 1413895071 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Bucket Columns: [] Sort Columns: [] Storage Desc Params: field.delim | - serialization.format | \ No newline at end of file + serialization.format | diff --git a/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 b/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 index 1d5891034ddec..6489eb562dbf7 100644 --- a/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 +++ b/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 @@ -1,3 +1,3 @@ tst1 500 tst2 500 -tst3 500 \ No newline at end of file +tst3 500 diff --git a/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f b/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f index 1ec0096b865a7..c86307863b92d 100644 --- a/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f +++ b/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f @@ -1,3 +1,3 @@ tst1 1 tst2 1 -tst3 1 \ No newline at end of file +tst3 1 diff --git a/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 b/sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 b/sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 b/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 index 2209b11d74282..f085b06e588d4 100644 --- a/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 +++ b/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 @@ -997,4 +997,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 b/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 index 07f67df79dcbd..5ba4e492aa0f4 100644 --- a/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 +++ b/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 @@ -14,4 +14,4 @@ NULL 10 406 1 66 1 98 1 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 b/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 index b00b55f6fa92d..20cb88d87f6bf 100644 --- a/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 +++ b/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 @@ -14,4 +14,4 @@ NULL 20 406 2 66 2 98 2 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 b/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 index 6f680072350ab..eeb33ca418288 100644 --- a/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 +++ b/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 @@ -1 +1 @@ -12500 \ No newline at end of file +12500 diff --git a/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 index a5a9e42fff209..4d199c29dd4db 100644 --- a/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -307,4 +307,4 @@ 96 1 97 1 98 1 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 index be13b26eadc86..5b6f2e2d4f773 100644 --- a/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 @@ -307,4 +307,4 @@ 96 val_96 1 97 val_97 1 98 val_98 1 -tst1 500 1 \ No newline at end of file +tst1 500 1 diff --git a/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 b/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 index 3dee790d7c2ca..11a5a2c531a3d 100644 --- a/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 +++ b/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 @@ -498,4 +498,4 @@ 97 val_97 98 val_98 98 val_98 -tst1 500 \ No newline at end of file +tst1 500 diff --git a/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 b/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 index e438a64050723..b95429f0c39e8 100644 --- a/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 +++ b/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 @@ -498,4 +498,4 @@ 97 val_97 val_97 98 val_98 val_98 98 val_98 val_98 -tst1 500 500 \ No newline at end of file +tst1 500 500 diff --git a/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 b/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 index e4dfefc43e26a..f21cd3313298c 100644 --- a/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 +++ b/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 @@ -307,4 +307,4 @@ 96 1 97 2 98 2 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 b/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 index e438a64050723..b95429f0c39e8 100644 --- a/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 +++ b/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 @@ -498,4 +498,4 @@ 97 val_97 val_97 98 val_98 val_98 98 val_98 val_98 -tst1 500 500 \ No newline at end of file +tst1 500 500 diff --git a/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 b/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 index e37d32abba426..83b33d238dab9 100644 --- a/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 +++ b/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 @@ -1 +1 @@ -1000 \ No newline at end of file +1000 diff --git a/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c b/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c index 385b8df6703a0..f8539c4817e21 100644 --- a/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c +++ b/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c @@ -20,4 +20,4 @@ 5 val_5 5 val_5 5 val_5 5 val_5 9 val_9 9 val_9 -tst1 500 tst1 500 \ No newline at end of file +tst1 500 tst1 500 diff --git a/sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c b/sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 b/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 deleted file mode 100644 index 26a2aff52e9a9..0000000000000 --- a/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 +++ /dev/null @@ -1,536 +0,0 @@ -NULL 2 -0 7 -001 2 -002 2 -004 1 -01 1 -013 1 -02 1 -021 2 -03 1 -032 5 -034 3 -051 1 -061 1 -062 1 -063 1 -064 1 -07 3 -071 1 -074 1 -08 1 -081 1 -082 2 -084 3 -09 3 -091 1 -094 1 -1 500 -10 2 -100 2 -102 1 -103 2 -104 7 -105 1 -11 2 -111 2 -113 5 -114 2 -116 1 -118 2 -119 3 -12 2 -120 2 -122 2 -123 2 -124 1 -125 2 -126 1 -128 3 -129 2 -131 2 -133 3 -134 5 -136 1 -137 2 -138 4 -14 1 -142 1 -143 2 -145 1 -146 2 -149 2 -15 4 -150 1 -152 2 -153 2 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 2 -165 2 -166 1 -167 3 -168 1 -169 4 -17 1 -170 1 -172 2 -174 2 -175 2 -176 2 -177 1 -178 1 -179 2 -18 2 -180 1 -181 2 -182 2 -183 1 -184 1 -186 1 -187 3 -189 1 -19 1 -190 1 -191 4 -192 2 -193 3 -194 2 -195 2 -196 1 -197 2 -199 3 -2 2 -20 2 -200 2 -201 1 -202 2 -203 3 -204 1 -205 2 -207 2 -208 3 -209 2 -21 2 -213 2 -214 1 -216 2 -217 2 -218 1 -219 2 -221 2 -222 2 -223 4 -224 2 -226 1 -228 1 -229 2 -230 5 -233 3 -234 1 -235 1 -237 2 -238 2 -239 2 -24 4 -241 1 -242 4 -243 2 -244 1 -247 1 -248 1 -249 1 -251 2 -252 2 -254 1 -255 2 -256 2 -257 1 -258 1 -26 2 -260 1 -261 1 -262 2 -263 2 -264 2 -265 2 -266 1 -27 3 -271 2 -272 4 -273 3 -274 2 -275 1 -277 4 -278 2 -28 2 -280 2 -281 2 -282 4 -283 3 -284 2 -285 1 -286 1 -287 1 -288 2 -289 1 -29 1 -291 2 -292 2 -293 1 -294 2 -296 1 -298 3 -30 2 -301 2 -302 3 -304 3 -305 1 -306 1 -307 2 -308 1 -309 2 -310 1 -311 5 -312 2 -314 2 -315 1 -316 3 -317 2 -318 3 -321 2 -322 4 -323 2 -325 2 -327 3 -33 2 -331 3 -332 3 -333 4 -335 1 -336 1 -338 1 -339 1 -34 2 -341 2 -342 2 -344 3 -345 1 -348 5 -35 4 -351 2 -353 4 -354 1 -356 1 -360 1 -361 1 -362 2 -364 3 -365 1 -366 1 -367 2 -368 1 -369 3 -37 2 -372 3 -373 2 -374 1 -375 1 -377 1 -378 1 -379 1 -38 2 -381 1 -382 3 -384 4 -386 1 -389 1 -391 3 -392 1 -393 2 -394 2 -395 2 -396 3 -397 2 -399 2 -4 2 -40 1 -400 1 -401 7 -402 1 -403 3 -404 4 -406 4 -407 1 -409 3 -41 1 -411 2 -412 1 -413 2 -414 4 -417 3 -418 1 -419 1 -42 4 -421 1 -422 2 -424 4 -427 1 -429 2 -43 2 -430 3 -431 5 -432 1 -435 1 -436 1 -437 1 -438 3 -439 2 -44 2 -442 1 -443 3 -444 2 -446 1 -448 1 -449 1 -45 1 -452 1 -453 1 -454 6 -455 1 -457 1 -458 2 -459 2 -46 1 -460 1 -461 2 -462 2 -463 3 -466 3 -467 1 -468 4 -469 5 -47 2 -470 1 -471 2 -472 2 -473 1 -475 1 -477 1 -478 2 -479 1 -48 2 -480 3 -481 1 -482 2 -483 4 -484 2 -485 1 -487 1 -489 4 -490 1 -491 2 -492 2 -493 2 -494 2 -495 1 -496 1 -497 1 -498 3 -5 6 -50 1 -501 1 -502 2 -503 1 -51 4 -513 1 -521 2 -523 2 -53 4 -532 1 -533 1 -534 1 -54 1 -541 1 -543 1 -551 1 -552 2 -554 1 -56 1 -561 2 -562 2 -563 1 -57 1 -571 2 -572 1 -573 1 -574 1 -58 3 -582 1 -584 1 -59 2 -591 2 -593 2 -594 1 -60 1 -603 1 -604 4 -611 1 -612 2 -613 3 -62 2 -621 1 -622 1 -631 1 -633 1 -634 1 -64 1 -641 2 -644 1 -65 1 -651 1 -652 2 -653 1 -66 2 -661 1 -662 1 -663 1 -664 3 -67 4 -671 2 -68 1 -681 1 -682 1 -683 1 -69 2 -691 1 -692 1 -693 3 -694 1 -70 4 -702 2 -703 2 -704 1 -71 1 -712 2 -713 2 -714 3 -72 3 -723 3 -724 1 -73 2 -731 2 -732 2 -734 1 -74 2 -742 1 -75 1 -751 1 -752 1 -754 1 -76 4 -761 3 -763 2 -764 1 -77 2 -771 1 -772 4 -773 1 -774 1 -78 2 -781 3 -782 1 -784 1 -79 2 -791 2 -793 2 -794 1 -8 2 -80 2 -802 3 -803 1 -81 2 -811 2 -812 1 -813 3 -814 1 -82 2 -821 3 -822 1 -83 2 -831 4 -832 2 -833 1 -834 3 -84 2 -842 1 -843 5 -844 1 -85 3 -851 1 -852 1 -854 2 -86 1 -861 1 -863 1 -864 4 -87 2 -871 1 -872 2 -873 1 -874 2 -882 2 -89 2 -892 3 -894 3 -9 2 -90 4 -902 2 -903 2 -904 3 -91 1 -911 3 -912 2 -914 1 -92 1 -921 2 -922 2 -924 2 -932 2 -933 1 -934 2 -941 2 -942 1 -944 1 -95 2 -954 2 -96 2 -961 4 -963 3 -964 5 -97 2 -971 2 -973 1 -974 1 -98 2 -981 1 -982 1 -983 1 -984 4 -991 3 -993 2 -record_0 1 -record_1 1 -record_2 1 -record_3 1 -record_4 1 -record_5 1 -record_6 1 -record_7 1 -record_8 1 -record_9 1 diff --git a/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd b/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd index bf0f76662bd2c..14ac2a0543eb7 100644 --- a/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd +++ b/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd @@ -997,4 +997,4 @@ 98 val_98 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 b/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 index 199095f1f9848..61d7280758de9 100644 --- a/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 +++ b/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 @@ -5,4 +5,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 index 293f324297419..0e14af56f8b08 100644 --- a/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 +++ b/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 @@ -17,4 +17,4 @@ 8 val_8 8 val_8 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 index 87a971baed428..ba254c7ab023b 100644 --- a/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 +++ b/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 @@ -17,4 +17,4 @@ 5 val_5 5 val_5 5 val_5 -5 val_5 \ No newline at end of file +5 val_5 diff --git a/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 index a6f502e6c3e3a..996f02bc6496c 100644 --- a/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 +++ b/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 @@ -17,4 +17,4 @@ 5 val_5 5 val_5 5 val_5 -5 val_5 \ No newline at end of file +5 val_5 diff --git a/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 b/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 index dadf53962e7a1..b1fb9cce36fa2 100644 --- a/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 +++ b/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 @@ -3,4 +3,4 @@ 4 2 5 6 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e b/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e index dadf53962e7a1..b1fb9cce36fa2 100644 --- a/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e +++ b/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e @@ -3,4 +3,4 @@ 4 2 5 6 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 b/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 index ede645acaf95b..48255af904349 100644 --- a/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 +++ b/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 @@ -3,4 +3,4 @@ 4 2 5 2 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 b/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 index ede645acaf95b..48255af904349 100644 --- a/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 +++ b/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 @@ -3,4 +3,4 @@ 4 2 5 2 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb b/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb index dadf53962e7a1..b1fb9cce36fa2 100644 --- a/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb +++ b/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb @@ -3,4 +3,4 @@ 4 2 5 6 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 b/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 index a71793e8c52a3..c7f78ecade3e8 100644 --- a/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 +++ b/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 @@ -3,4 +3,4 @@ val_2 2 val_4 2 val_5 6 val_8 2 -val_9 2 \ No newline at end of file +val_9 2 diff --git a/sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e b/sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 b/sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b b/sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 b/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 deleted file mode 100644 index 9bf0de2f06c9a..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 +++ /dev/null @@ -1,32 +0,0 @@ -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -2.0 2.0 -2.0 2.0 -4.0 4.0 -4.0 4.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -8.0 8.0 -8.0 8.0 -9.0 9.0 -9.0 9.0 diff --git a/sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 b/sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 b/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 deleted file mode 100644 index 462dca3124c41..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 +++ /dev/null @@ -1,20 +0,0 @@ -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -2.0 -2.0 -4.0 -4.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -8.0 -8.0 -9.0 -9.0 diff --git a/sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e b/sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 b/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 deleted file mode 100644 index 2e662f39d1572..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 +++ /dev/null @@ -1,32 +0,0 @@ -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -2.0 -2.0 -4.0 -4.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -8.0 -8.0 -9.0 -9.0 diff --git a/sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b b/sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 b/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 deleted file mode 100644 index 2e662f39d1572..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 +++ /dev/null @@ -1,32 +0,0 @@ -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -2.0 -2.0 -4.0 -4.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -8.0 -8.0 -9.0 -9.0 diff --git a/sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb b/sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db b/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db deleted file mode 100644 index 106f1874451ff..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db +++ /dev/null @@ -1,32 +0,0 @@ -0.0 0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -2.0 2.0 -2.0 2 -4.0 4 -4.0 4.0 -5.0 5 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5 -5.0 5.0 -5.0 5.0 -5.0 5.0 -8.0 8.0 -8.0 8 -9.0 9 -9.0 9.0 diff --git a/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 b/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 index d572335ff5185..7369ee2317540 100644 --- a/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 +++ b/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 @@ -27,4 +27,4 @@ 86 val_86 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 b/sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c b/sql/hive/src/test/resources/golden/union34-4-70479e10c016e5ac448394dbadb32794 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c rename to sql/hive/src/test/resources/golden/union34-4-70479e10c016e5ac448394dbadb32794 diff --git a/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 b/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 index d572335ff5185..7369ee2317540 100644 --- a/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 +++ b/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 @@ -27,4 +27,4 @@ 86 val_86 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 b/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 index 948aca9180ba9..abc382d9b3571 100644 --- a/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 +++ b/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 @@ -1,2 +1,2 @@ tst1 500 -tst2 500 \ No newline at end of file +tst2 500 diff --git a/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 b/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 index c0c7d9f5e8ef5..9daa3f92b46e0 100644 --- a/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 +++ b/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 @@ -1,2 +1,2 @@ tst1 1 -tst2 1 \ No newline at end of file +tst2 1 diff --git a/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 b/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 index b5e91e032212a..6f79451ce69b5 100644 --- a/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 +++ b/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 @@ -23,4 +23,4 @@ NULL val_484 406 val_406 66 val_66 98 val_98 -tst1 500 \ No newline at end of file +tst1 500 diff --git a/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 b/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 index 07f67df79dcbd..5ba4e492aa0f4 100644 --- a/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 +++ b/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 @@ -14,4 +14,4 @@ NULL 10 406 1 66 1 98 1 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 b/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 index 3617909902993..52982787dbb4a 100644 --- a/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 +++ b/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 @@ -1497,4 +1497,4 @@ 200 val_200 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 b/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 index 37021f4a27201..3d86ec6498f3f 100644 --- a/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 +++ b/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 @@ -1 +1 @@ -1500 \ No newline at end of file +1500 diff --git a/sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c b/sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 b/sql/hive/src/test/resources/golden/union_date-4-d85fe746334b430941c5db3665e744d4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 rename to sql/hive/src/test/resources/golden/union_date-4-d85fe746334b430941c5db3665e744d4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 b/sql/hive/src/test/resources/golden/union_date-5-82eebfded24cef08e0a881d1bcca02b1 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 rename to sql/hive/src/test/resources/golden/union_date-5-82eebfded24cef08e0a881d1bcca02b1 diff --git a/sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d b/sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a b/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a deleted file mode 100644 index 468f774ef5146..0000000000000 --- a/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a +++ /dev/null @@ -1,10 +0,0 @@ -val_238 -NULL -val_86 -NULL -val_311 -NULL -val_27 -NULL -val_165 -NULL diff --git a/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index c203753d8e42d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:49:42 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069004 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 b/sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 b/sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 b/sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 94a711cdab117..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:50:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069036 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 b/sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 +++ b/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c +++ b/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 index d9cfa9e952c3a..4c47424eb1160 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ b/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 @@ -1,19 +1,24 @@ # col_name data_type comment -key string from deserializer -values bigint from deserializer +key string +values bigint # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 19:05:51 PST 2014 +CreateTime: Tue Oct 21 05:59:44 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/outputtbl1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/outputtbl1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388804758 + COLUMN_STATS_ACCURATE false + numFiles 1 + numRows -1 + rawDataSize -1 + totalSize 184 + transient_lastDdlTime 1413896391 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe @@ -24,4 +29,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e index 2817d74afee13..8930b40bed84f 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e +++ b/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e @@ -15,4 +15,4 @@ 8 2 8 2 8 3 -8 3 \ No newline at end of file +8 3 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a +++ b/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 +++ b/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc +++ b/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 +++ b/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 b/sql/hive/src/test/resources/golden/union_remove_11-9-94da21f150ed2c56046b80e46da8884d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 rename to sql/hive/src/test/resources/golden/union_remove_11-9-94da21f150ed2c56046b80e46da8884d diff --git a/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c b/sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 b/sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 57d5146698ee7..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:50:41 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069061 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 b/sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 b/sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af b/sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index c914024c14c8e..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:51:05 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069086 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb b/sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c b/sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 b/sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 5a55428afbe1a..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:51:30 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069108 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d b/sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 b/sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 b/sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index a2441e330ffc6..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:51:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069112 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 deleted file mode 100644 index a58273a1b054e..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 +++ /dev/null @@ -1,2 +0,0 @@ -ds=1 -ds=2 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d b/sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 b/sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 b/sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index bec8202dcbcad..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:52:19 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069139 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 deleted file mode 100644 index a58273a1b054e..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 +++ /dev/null @@ -1,2 +0,0 @@ -ds=1 -ds=2 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a b/sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 b/sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d b/sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index d0b37e7625a64..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 14 14:38:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4075462935071533647/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1389739098 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 deleted file mode 100644 index c21f4017362c1..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 +++ /dev/null @@ -1,2 +0,0 @@ -ds=1 -ds=2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 b/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 deleted file mode 100644 index 1ab5c482feca9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 +++ /dev/null @@ -1,6 +0,0 @@ -1 1 1 -2 1 1 -3 1 1 -7 1 1 -8 1 1 -8 1 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a b/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a deleted file mode 100644 index 94ec062e2f4f3..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a +++ /dev/null @@ -1,6 +0,0 @@ -1 2 2 -2 2 2 -3 2 2 -7 2 2 -8 2 2 -8 2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b b/sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad b/sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 b/sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 971e095df7be8..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:52:47 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069167 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 deleted file mode 100644 index 5690d94a65857..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 +++ /dev/null @@ -1,6 +0,0 @@ -ds=11 -ds=12 -ds=13 -ds=17 -ds=18 -ds=28 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 b/sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 b/sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b b/sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 0d0d3a30af36c..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:53:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069215 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 deleted file mode 100644 index 719e4a7bd4dbb..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 b/sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 b/sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 deleted file mode 100644 index 361f2ab78eae3..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 +++ /dev/null @@ -1,2 +0,0 @@ -7 1 -7 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e b/sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 b/sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 b/sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 b/sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index d3c821fa22182..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:53:36 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069242 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e b/sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 b/sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 978049640929c..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -values bigint None -key string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:54:06 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069268 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e b/sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f b/sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa b/sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 b/sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index ce7e0438f9eff..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,26 +0,0 @@ -# col_name data_type comment - -key string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:54:32 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069293 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f b/sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 b/sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 b/sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 b/sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index c261ce48bdfb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,28 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None -values2 bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:54:58 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069319 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 deleted file mode 100644 index d36ac6b834792..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 +++ /dev/null @@ -1,10 +0,0 @@ -1 1 1 -2 1 1 -3 1 1 -7 1 1 -8 2 2 -1 1 1 -2 1 1 -3 1 1 -7 1 1 -8 2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd b/sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af b/sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 b/sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 b/sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb b/sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 b/sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 50297abd4e9a6..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:55:20 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069345 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 b/sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 b/sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 b/sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index f3c452ebe9284..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key double None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:55:49 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069371 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 b/sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e b/sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f b/sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 +++ b/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c +++ b/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 index d46af80f0d1ae..8cd7d42b20df5 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ b/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -1,19 +1,24 @@ # col_name data_type comment -key string None -values bigint None +key string +values bigint # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 19:09:55 PST 2014 +CreateTime: Tue Oct 21 06:00:02 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/outputtbl1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/outputtbl1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388805002 + COLUMN_STATS_ACCURATE false + numFiles 1 + numRows -1 + rawDataSize -1 + totalSize 72 + transient_lastDdlTime 1413896409 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +29,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e index 2817d74afee13..8930b40bed84f 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e +++ b/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e @@ -15,4 +15,4 @@ 8 2 8 2 8 3 -8 3 \ No newline at end of file +8 3 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 +++ b/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/union_remove_3-8-94da21f150ed2c56046b80e46da8884d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/union_remove_3-8-94da21f150ed2c56046b80e46da8884d diff --git a/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 67faa5b13d984..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:56:16 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead b/sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 19aeb0b602859..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:56:42 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069430 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 b/sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 +++ b/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c +++ b/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e index 8e7ee8a2b47bb..b4ea93dc7dd89 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e +++ b/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e @@ -7,4 +7,4 @@ 7 1 7 1 8 2 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 b/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 index 8e7ee8a2b47bb..b4ea93dc7dd89 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 +++ b/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 @@ -7,4 +7,4 @@ 7 1 7 1 8 2 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 +++ b/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 b/sql/hive/src/test/resources/golden/union_remove_6-9-94da21f150ed2c56046b80e46da8884d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 rename to sql/hive/src/test/resources/golden/union_remove_6-9-94da21f150ed2c56046b80e46da8884d diff --git a/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 1c63b3851aad2..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:57:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069463 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca b/sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index f718ab2990cdd..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:57:48 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069495 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee b/sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 b/sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c b/sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 65a91a6027545..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:58:19 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069518 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 b/sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 b/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 index 42d07ed10bad4..760e62cce0bba 100644 --- a/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 +++ b/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 @@ -497,4 +497,4 @@ 97 97 98 -98 \ No newline at end of file +98 diff --git a/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 b/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 index 2f23db6a2ff4a..3bafdf359f6fa 100644 --- a/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 +++ b/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 @@ -997,4 +997,4 @@ 98 98 98 -98 \ No newline at end of file +98 diff --git a/sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d b/sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 b/sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 b/sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 b/sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e b/sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 b/sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae b/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 b/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da b/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da index 08661a3d7f794..6fc243e718c1b 100644 --- a/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da +++ b/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da @@ -2,4 +2,4 @@ val_0 0 3 val_10 10 1 val_100 200 2 val_103 206 2 -val_104 208 2 \ No newline at end of file +val_104 208 2 diff --git a/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f b/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f index 08661a3d7f794..6fc243e718c1b 100644 --- a/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f +++ b/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f @@ -2,4 +2,4 @@ val_0 0 3 val_10 10 1 val_100 200 2 val_103 206 2 -val_104 208 2 \ No newline at end of file +val_104 208 2 diff --git a/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 b/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 index f0c744ca37d95..0052fe0c959c5 100644 --- a/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 +++ b/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 @@ -2,4 +2,4 @@ val_98 196 2 val_97 194 2 val_96 96 1 val_95 190 2 -val_92 92 1 \ No newline at end of file +val_92 92 1 diff --git a/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f b/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f index f0c744ca37d95..0052fe0c959c5 100644 --- a/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f +++ b/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f @@ -2,4 +2,4 @@ val_98 196 2 val_97 194 2 val_96 96 1 val_95 190 2 -val_92 92 1 \ No newline at end of file +val_92 92 1 diff --git a/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce b/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce index 35e7795b9c2a7..fa55ed96ee0b9 100644 --- a/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce +++ b/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce @@ -1,3 +1,3 @@ 1 abc 1 abc 2 abc 2 abc -3 abc 3 abc \ No newline at end of file +3 abc 3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c b/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c index 35e7795b9c2a7..fa55ed96ee0b9 100644 --- a/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c +++ b/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c @@ -1,3 +1,3 @@ 1 abc 1 abc 2 abc 2 abc -3 abc 3 abc \ No newline at end of file +3 abc 3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef b/sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 b/sql/hive/src/test/resources/golden/varchar_join1-6-ab4392aa5ff499ec43229425ff23e22f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 rename to sql/hive/src/test/resources/golden/varchar_join1-6-ab4392aa5ff499ec43229425ff23e22f diff --git a/sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 b/sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 b/sql/hive/src/test/resources/golden/varchar_join1-7-a01639290aaceb4b85aa6e44319f6386 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 rename to sql/hive/src/test/resources/golden/varchar_join1-7-a01639290aaceb4b85aa6e44319f6386 diff --git a/sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 b/sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 b/sql/hive/src/test/resources/golden/varchar_join1-8-c05d7b534b51cecdc2ba2de4ce57ba37 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 rename to sql/hive/src/test/resources/golden/varchar_join1-8-c05d7b534b51cecdc2ba2de4ce57ba37 diff --git a/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 b/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 index 35e7795b9c2a7..fa55ed96ee0b9 100644 --- a/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 +++ b/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 @@ -1,3 +1,3 @@ 1 abc 1 abc 2 abc 2 abc -3 abc 3 abc \ No newline at end of file +3 abc 3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc b/sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 b/sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 b/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 deleted file mode 100644 index 05066db296756..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 +++ /dev/null @@ -1 +0,0 @@ -c1 array None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 b/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 deleted file mode 100644 index 05cd8fccf276c..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 +++ /dev/null @@ -1 +0,0 @@ -["val_0","val_0"] diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f b/sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 b/sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 b/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 deleted file mode 100644 index 1ec6828af9fd2..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 +++ /dev/null @@ -1 +0,0 @@ -c1 map None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 b/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 deleted file mode 100644 index 8f12062de4e27..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 +++ /dev/null @@ -1 +0,0 @@ -{0:"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 b/sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc b/sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 b/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 deleted file mode 100644 index 030d9d6ec0429..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 +++ /dev/null @@ -1 +0,0 @@ -c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 b/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 deleted file mode 100644 index f350c1112ac2c..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 +++ /dev/null @@ -1 +0,0 @@ -{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb b/sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 b/sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad b/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad deleted file mode 100644 index 030d9d6ec0429..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad +++ /dev/null @@ -1 +0,0 @@ -c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 b/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 deleted file mode 100644 index f350c1112ac2c..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 +++ /dev/null @@ -1 +0,0 @@ -{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 b/sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 b/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 deleted file mode 100644 index 030d9d6ec0429..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 +++ /dev/null @@ -1 +0,0 @@ -c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb b/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb deleted file mode 100644 index f350c1112ac2c..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb +++ /dev/null @@ -1 +0,0 @@ -{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc b/sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 b/sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb b/sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 b/sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 b/sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 b/sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 b/sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 b/sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 b/sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b b/sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 b/sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 b/sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 b/sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 b/sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d b/sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 b/sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e b/sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 b/sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e b/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 b/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 b/sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 b/sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a b/sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d b/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad b/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e b/sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 b/sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b b/sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 b/sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 b/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 b/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc b/sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f b/sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 b/sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e b/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c b/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 b/sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a b/sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 b/sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb b/sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 b/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 b/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 b/sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d b/sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e b/sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 b/sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f b/sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f b/sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba b/sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba b/sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 b/sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 b/sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 b/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 b/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 b/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 index 173eb955a485a..52bb2199e16a3 100644 --- a/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 +++ b/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 @@ -1,4 +1,4 @@ 1 abc 1 abc 2 abc -3 abc \ No newline at end of file +3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 b/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 index 173eb955a485a..52bb2199e16a3 100644 --- a/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 +++ b/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 @@ -1,4 +1,4 @@ 1 abc 1 abc 2 abc -3 abc \ No newline at end of file +3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 b/sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/varchar_union1-6-f338f341c5f86d0a44cabfb4f7bddc3b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b rename to sql/hive/src/test/resources/golden/varchar_union1-6-f338f341c5f86d0a44cabfb4f7bddc3b diff --git a/sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 b/sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc b/sql/hive/src/test/resources/golden/varchar_union1-7-ea0d1fbae997b50dc34f7610480bbe29 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc rename to sql/hive/src/test/resources/golden/varchar_union1-7-ea0d1fbae997b50dc34f7610480bbe29 diff --git a/sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 b/sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e b/sql/hive/src/test/resources/golden/varchar_union1-8-f3be9a2498927d692356c2cf871d25bf similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e rename to sql/hive/src/test/resources/golden/varchar_union1-8-f3be9a2498927d692356c2cf871d25bf diff --git a/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 b/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 index 173eb955a485a..52bb2199e16a3 100644 --- a/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 +++ b/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 @@ -1,4 +1,4 @@ 1 abc 1 abc 2 abc -3 abc \ No newline at end of file +3 abc diff --git a/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 b/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 b/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 b/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 b/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 b/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee b/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b b/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 b/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a b/sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 b/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b b/sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 b/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 b/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 b/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 b/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf b/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb b/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 b/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de b/sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 b/sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 b/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a b/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 b/sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb b/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 b/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 b/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 b/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe b/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d b/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d deleted file mode 100644 index ea00577174e43..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d +++ /dev/null @@ -1,500 +0,0 @@ -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 238 0 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 86 12 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 311 22 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 27 34 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 165 44 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 409 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 255 68 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 278 80 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 98 92 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 484 102 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 265 114 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 193 126 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 138 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 150 150 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 273 162 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 224 174 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 369 186 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 66 198 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 128 208 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 213 220 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 146 232 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 244 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 429 256 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 374 268 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 152 280 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 292 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 145 304 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 495 316 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 37 328 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 327 338 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 281 350 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 362 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 209 374 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 15 386 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 82 396 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 403 406 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 166 418 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 417 430 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 430 442 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 252 454 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 292 466 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 219 478 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 287 490 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 153 502 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 193 514 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 338 526 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 446 538 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 459 550 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 394 562 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 237 574 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 482 586 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 174 598 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 413 610 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 494 622 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 207 634 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 199 646 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 466 658 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 208 670 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 174 682 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 399 694 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 396 706 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 247 718 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 417 730 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 742 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 162 754 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 377 766 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 397 778 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 309 790 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 365 802 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 266 814 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 439 826 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 342 838 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 367 850 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 325 862 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 167 874 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 195 886 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 475 898 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 17 910 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 113 920 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 155 932 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 203 944 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 339 956 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 0 968 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 455 976 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 128 988 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 311 1000 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 316 1012 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 57 1024 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 302 1034 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 205 1046 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 149 1058 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 438 1070 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 345 1082 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 129 1094 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 170 1106 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 20 1118 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 1128 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 157 1140 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 378 1152 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 221 1164 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 92 1176 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 111 1186 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 47 1198 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 72 1208 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 4 1218 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 280 1226 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 35 1238 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 427 1248 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 1260 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 208 1272 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 356 1284 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 399 1296 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 1308 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 382 1320 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 498 1332 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 125 1344 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 386 1356 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 437 1368 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 1380 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 192 1392 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 286 1404 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 187 1416 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 176 1428 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 54 1440 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 459 1450 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 51 1462 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 1472 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 103 1484 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 239 1496 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 213 1508 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 216 1520 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 430 1532 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 278 1544 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 176 1556 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 289 1568 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 221 1580 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 65 1592 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 318 1602 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 332 1614 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 311 1626 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 275 1638 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 137 1650 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 241 1662 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 83 1674 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 333 1684 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 180 1696 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 284 1708 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 12 1720 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 1730 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 181 1742 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 67 1754 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 260 1764 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 404 1776 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 384 1788 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 1800 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 353 1812 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 373 1824 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 272 1836 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 1848 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 217 1860 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 84 1872 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 1882 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 466 1894 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 58 1906 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 8 1916 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 411 1924 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 1936 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 208 1948 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 1960 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 24 1972 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 463 1982 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 431 1994 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 179 2006 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 172 2018 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 42 2030 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 129 2040 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 158 2052 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 119 2064 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 496 2076 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 0 2088 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 322 2096 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 197 2108 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 2120 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 393 2132 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 454 2144 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 100 2156 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 298 2168 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 199 2180 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 191 2192 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 418 2204 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 96 2216 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 26 2226 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 165 2236 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 327 2248 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 2260 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 205 2272 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 120 2284 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 131 2296 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 51 2308 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 404 2318 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 43 2330 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 436 2340 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 156 2352 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 2364 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 2376 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 308 2388 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 95 2400 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 196 2410 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 288 2422 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 481 2434 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 457 2446 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 98 2458 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 282 2468 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 197 2480 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 187 2492 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 318 2504 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 318 2516 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 409 2528 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 470 2540 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 137 2552 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 369 2564 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 316 2576 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 2588 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 413 2600 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 85 2612 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 77 2622 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 0 2632 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 490 2640 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 87 2652 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 364 2662 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 179 2674 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 118 2686 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 134 2698 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 395 2710 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 282 2722 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 2734 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 238 2746 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 419 2758 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 15 2770 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 118 2780 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 72 2792 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 90 2802 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 307 2812 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 19 2824 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 435 2834 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 10 2846 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 2856 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 273 2868 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 306 2880 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 224 2892 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 309 2904 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 389 2916 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 327 2928 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 242 2940 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 369 2952 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 392 2964 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 272 2976 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 331 2988 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 3000 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 242 3012 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 452 3024 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 177 3036 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 226 3048 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 5 3060 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 497 3068 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 402 3080 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 396 3092 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 317 3104 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 395 3116 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 58 3128 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 35 3138 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 336 3148 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 95 3160 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 11 3170 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 168 3180 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 34 3192 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 229 3202 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 233 3214 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 143 3226 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 472 3238 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 322 3250 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 498 3262 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 160 3274 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 195 3286 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 42 3298 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 321 3308 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 430 3320 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 119 3332 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 3344 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 458 3356 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 78 3368 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 76 3378 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 41 3388 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 223 3398 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 492 3410 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 149 3422 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 449 3434 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 218 3446 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 228 3458 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 3470 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 453 3482 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 30 3494 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 209 3504 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 64 3516 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 3526 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 76 3538 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 74 3548 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 342 3558 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 69 3570 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 3580 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 33 3592 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 368 3602 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 103 3614 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 296 3626 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 113 3638 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 216 3650 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 367 3662 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 344 3674 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 167 3686 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 274 3698 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 219 3710 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 239 3722 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 485 3734 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 116 3746 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 223 3758 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 256 3770 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 263 3782 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 70 3794 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 487 3804 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 480 3816 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 3828 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 288 3840 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 191 3852 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 5 3864 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 244 3872 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 438 3884 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 128 3896 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 467 3908 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 432 3920 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 202 3932 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 316 3944 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 229 3956 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 3968 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 463 3980 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 280 3992 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 2 4004 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 35 4012 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 283 4022 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 331 4034 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 235 4046 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 80 4058 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 44 4068 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 193 4078 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 321 4090 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 335 4102 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 104 4114 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 466 4126 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 366 4138 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 175 4150 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 403 4162 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 483 4174 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 53 4186 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 105 4196 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 257 4208 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 4220 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 409 4232 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 190 4244 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 4256 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 4268 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 114 4280 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 258 4292 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 90 4304 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 203 4314 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 262 4326 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 4338 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 424 4350 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 12 4362 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 396 4372 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 201 4384 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 217 4396 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 164 4408 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 431 4420 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 454 4432 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 478 4444 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 298 4456 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 125 4468 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 431 4480 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 164 4492 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 424 4504 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 187 4516 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 382 4528 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 5 4540 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 70 4548 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 397 4558 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 480 4570 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 291 4582 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 24 4594 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 351 4604 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 255 4616 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 104 4628 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 70 4640 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 163 4650 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 438 4662 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 119 4674 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 414 4686 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 200 4698 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 491 4710 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 237 4722 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 439 4734 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 360 4746 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 248 4758 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 479 4770 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 305 4782 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 417 4794 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 199 4806 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 444 4818 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 120 4830 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 429 4842 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 443 4866 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 323 4878 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 325 4890 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 4902 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 4914 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 478 4926 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 178 4938 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 4950 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 310 4962 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 317 4974 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 333 4986 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 493 4998 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 460 5010 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 207 5022 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 249 5034 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 265 5046 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 480 5058 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 83 5070 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 136 5080 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 353 5092 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 172 5104 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 214 5116 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 462 5128 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 233 5140 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 5152 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 133 5164 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 175 5176 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 189 5188 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 454 5200 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 375 5212 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 5224 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 421 5236 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 407 5248 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 384 5260 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 256 5272 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 26 5284 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 134 5294 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 67 5306 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 384 5316 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 379 5328 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 18 5340 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 462 5350 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 492 5362 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 100 5374 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 298 5386 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 9 5398 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 341 5406 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 498 5418 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 146 5430 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 458 5442 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 362 5454 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 186 5466 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 285 5478 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 5490 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 167 5502 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 18 5514 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 273 5524 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 183 5536 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 281 5548 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 344 5560 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 97 5572 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 5582 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 315 5594 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 84 5606 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 28 5616 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 37 5626 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 448 5636 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 152 5648 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 5660 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 307 5672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 194 5684 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 414 5696 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 477 5708 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 222 5720 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 126 5732 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 90 5744 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 5754 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 403 5766 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 400 5778 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 200 5790 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 97 5802 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 b/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 deleted file mode 100644 index df07a9da29f01..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 +++ /dev/null @@ -1,309 +0,0 @@ -0 3 -2 1 -4 1 -5 3 -8 1 -9 1 -10 1 -11 1 -12 2 -15 2 -17 1 -18 2 -19 1 -20 1 -24 2 -26 2 -27 1 -28 1 -30 1 -33 1 -34 1 -35 3 -37 2 -41 1 -42 2 -43 1 -44 1 -47 1 -51 2 -53 1 -54 1 -57 1 -58 2 -64 1 -65 1 -66 1 -67 2 -69 1 -70 3 -72 2 -74 1 -76 2 -77 1 -78 1 -80 1 -82 1 -83 2 -84 2 -85 1 -86 1 -87 1 -90 3 -92 1 -95 2 -96 1 -97 2 -98 2 -100 2 -103 2 -104 2 -105 1 -111 1 -113 2 -114 1 -116 1 -118 2 -119 3 -120 2 -125 2 -126 1 -128 3 -129 2 -131 1 -133 1 -134 2 -136 1 -137 2 -138 4 -143 1 -145 1 -146 2 -149 2 -150 1 -152 2 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 2 -165 2 -166 1 -167 3 -168 1 -169 4 -170 1 -172 2 -174 2 -175 2 -176 2 -177 1 -178 1 -179 2 -180 1 -181 1 -183 1 -186 1 -187 3 -189 1 -190 1 -191 2 -192 1 -193 3 -194 1 -195 2 -196 1 -197 2 -199 3 -200 2 -201 1 -202 1 -203 2 -205 2 -207 2 -208 3 -209 2 -213 2 -214 1 -216 2 -217 2 -218 1 -219 2 -221 2 -222 1 -223 2 -224 2 -226 1 -228 1 -229 2 -230 5 -233 2 -235 1 -237 2 -238 2 -239 2 -241 1 -242 2 -244 1 -247 1 -248 1 -249 1 -252 1 -255 2 -256 2 -257 1 -258 1 -260 1 -262 1 -263 1 -265 2 -266 1 -272 2 -273 3 -274 1 -275 1 -277 4 -278 2 -280 2 -281 2 -282 2 -283 1 -284 1 -285 1 -286 1 -287 1 -288 2 -289 1 -291 1 -292 1 -296 1 -298 3 -302 1 -305 1 -306 1 -307 2 -308 1 -309 2 -310 1 -311 3 -315 1 -316 3 -317 2 -318 3 -321 2 -322 2 -323 1 -325 2 -327 3 -331 2 -332 1 -333 2 -335 1 -336 1 -338 1 -339 1 -341 1 -342 2 -344 2 -345 1 -348 5 -351 1 -353 2 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 2 -368 1 -369 3 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 2 -384 3 -386 1 -389 1 -392 1 -393 1 -394 1 -395 2 -396 3 -397 2 -399 2 -400 1 -401 5 -402 1 -403 3 -404 2 -406 4 -407 1 -409 3 -411 1 -413 2 -414 2 -417 3 -418 1 -419 1 -421 1 -424 2 -427 1 -429 2 -430 3 -431 3 -432 1 -435 1 -436 1 -437 1 -438 3 -439 2 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 3 -455 1 -457 1 -458 2 -459 2 -460 1 -462 2 -463 2 -466 3 -467 1 -468 4 -469 5 -470 1 -472 1 -475 1 -477 1 -478 2 -479 1 -480 3 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 4 -490 1 -491 1 -492 2 -493 1 -494 1 -495 1 -496 1 -497 1 -498 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 b/sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 b/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 deleted file mode 100644 index 2a0cdc268347a..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 +++ /dev/null @@ -1,309 +0,0 @@ -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 0 [968,2632,2088] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 2 [4004] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 4 [1218] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 5 [4540,3864,3060] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 8 [1916] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 9 [5398] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 10 [2846] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 11 [3170] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 12 [4362,1720] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 15 [386,2770] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 17 [910] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 18 [5514,5340] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 19 [2824] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 20 [1118] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 24 [1972,4594] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 26 [5284,2226] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 27 [34] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 28 [5616] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 30 [3494] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 33 [3592] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 34 [3192] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 35 [1238,4012,3138] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 37 [328,5626] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 41 [3388] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 42 [3298,2030] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 43 [2330] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 44 [4068] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 47 [1198] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 51 [1462,2308] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 53 [4186] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 54 [1440] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 57 [1024] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 58 [3128,1906] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 64 [3516] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 65 [1592] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 66 [198] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 67 [5306,1754] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 69 [3570] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 70 [3794,4640,4548] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 72 [2792,1208] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 74 [3548] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 76 [3538,3378] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 77 [2622] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 78 [3368] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 80 [4058] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 82 [396] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 83 [5070,1674] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 84 [5606,1872] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 85 [2612] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 86 [12] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 87 [2652] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 90 [2802,5744,4304] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 92 [1176] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 95 [2400,3160] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 96 [2216] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 97 [5572,5802] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 98 [2458,92] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 100 [5374,2156] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 103 [3614,1484] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 104 [4628,4114] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 105 [4196] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 111 [1186] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 113 [920,3638] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 114 [4280] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 116 [3746] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 118 [2780,2686] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 119 [2064,4674,3332] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 120 [2284,4830] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 125 [4468,1344] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 126 [5732] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 128 [3896,988,208] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 129 [2040,1094] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 131 [2296] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 133 [5164] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 134 [2698,5294] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 136 [5080] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 137 [2552,1650] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 [2734,1848,1472,3470] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 143 [3226] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 145 [304] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 146 [232,5430] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 149 [3422,1058] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 150 [150] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 152 [280,5648] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 153 [502] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 155 [932] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 156 [2352] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 157 [1140] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 158 [2052] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 160 [3274] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 162 [754] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 163 [4650] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 164 [4492,4408] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 165 [2236,44] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 166 [418] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 167 [5502,874,3686] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 168 [3180] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 [5754,1308,2588,4854] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 170 [1106] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 172 [2018,5104] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 174 [682,598] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 175 [5176,4150] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 176 [1428,1556] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 177 [3036] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 178 [4938] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 179 [2674,2006] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 180 [1696] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 181 [1742] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 183 [5536] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 186 [5466] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 187 [2492,1416,4516] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 189 [5188] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 190 [4244] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 191 [3852,2192] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 192 [1392] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 193 [514,126,4078] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 194 [5684] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 195 [3286,886] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 196 [2410] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 197 [2480,2108] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 199 [646,2180,4806] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 200 [4698,5790] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 201 [4384] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 202 [3932] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 203 [4314,944] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 205 [1046,2272] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 207 [5022,634] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 208 [670,1948,1272] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 209 [3504,374] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 213 [220,1508] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 214 [5116] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 216 [1520,3650] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 217 [4396,1860] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 218 [3446] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 219 [478,3710] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 221 [1580,1164] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 222 [5720] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 223 [3398,3758] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 224 [174,2892] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 226 [3048] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 228 [3458] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 229 [3202,3956] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 [1730,1936,4914,2260,3580] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 233 [5140,3214] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 235 [4046] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 237 [4722,574] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 238 [0,2746] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 239 [1496,3722] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 241 [1662] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 242 [2940,3012] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 244 [3872] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 247 [718] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 248 [4758] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 249 [5034] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 252 [454] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 255 [68,4616] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 256 [5272,3770] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 257 [4208] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 258 [4292] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 260 [1764] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 262 [4326] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 263 [3782] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 265 [114,5046] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 266 [814] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 272 [1836,2976] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 273 [2868,5524,162] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 274 [3698] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 275 [1638] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 [4902,1260,2856,362] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 278 [1544,80] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 280 [3992,1226] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 281 [350,5548] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 282 [2468,2722] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 283 [4022] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 284 [1708] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 285 [5478] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 286 [1404] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 287 [490] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 288 [2422,3840] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 289 [1568] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 291 [4582] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 292 [466] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 296 [3626] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 298 [5386,4456,2168] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 302 [1034] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 305 [4782] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 306 [2880] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 307 [2812,5672] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 308 [2388] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 309 [790,2904] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 310 [4962] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 311 [1000,1626,22] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 315 [5594] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 316 [3944,2576,1012] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 317 [3104,4974] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 318 [2504,1602,2516] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 321 [4090,3308] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 322 [3250,2096] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 323 [4878] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 325 [4890,862] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 327 [2928,338,2248] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 331 [2988,4034] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 332 [1614] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 333 [1684,4986] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 335 [4102] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 336 [3148] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 338 [526] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 339 [956] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 341 [5406] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 342 [3558,838] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 344 [3674,5560] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 345 [1082] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 [5660,1882,1960,4338,5490] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 351 [4604] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 353 [1812,5092] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 356 [1284] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 360 [4746] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 362 [5454] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 364 [2662] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 365 [802] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 366 [4138] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 367 [850,3662] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 368 [3602] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 369 [186,2564,2952] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 373 [1824] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 374 [268] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 375 [5212] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 377 [766] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 378 [1152] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 379 [5328] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 382 [1320,4528] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 384 [5316,5260,1788] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 386 [1356] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 389 [2916] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 392 [2964] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 393 [2132] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 394 [562] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 395 [3116,2710] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 396 [4372,706,3092] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 397 [4558,778] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 399 [694,1296] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 400 [5778] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 [138,3000,3828,4268,5224] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 402 [3080] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 403 [5766,4162,406] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 404 [1776,2318] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 [244,4220,4256,5152] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 407 [5248] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 409 [4232,56,2528] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 411 [1924] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 413 [610,2600] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 414 [5696,4686] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 417 [730,4794,430] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 418 [2204] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 419 [2758] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 421 [5236] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 424 [4350,4504] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 427 [1248] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 429 [4842,256] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 430 [442,1532,3320] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 431 [4420,1994,4480] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 432 [3920] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 435 [2834] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 436 [2340] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 437 [1368] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 438 [3884,4662,1070] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 439 [4734,826] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 443 [4866] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 444 [4818] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 446 [538] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 448 [5636] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 449 [3434] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 452 [3024] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 453 [3482] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 454 [2144,4432,5200] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 455 [976] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 457 [2446] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 458 [3356,5442] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 459 [550,1450] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 460 [5010] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 462 [5350,5128] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 463 [1982,3980] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 466 [658,1894,4126] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 467 [3908] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 [3526,4950,2120,2376] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 [292,3968,1380,5582,2364] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 470 [2540] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 472 [3238] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 475 [898] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 477 [5708] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 478 [4444,4926] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 479 [4770] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 480 [4570,5058,3816] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 481 [2434] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 482 [586] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 483 [4174] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 484 [102] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 485 [3734] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 487 [3804] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 [1128,1800,3344,742] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 490 [2640] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 491 [4710] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 492 [5362,3410] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 493 [4998] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 494 [622] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 495 [316] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 496 [2076] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 497 [3068] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 498 [5418,3262,1332] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 b/sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 b/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 deleted file mode 100644 index 94754d8bd4c82..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 +++ /dev/null @@ -1 +0,0 @@ -97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 b/sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 b/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 b/sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 b/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 deleted file mode 100644 index e6902f60ff486..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 +++ /dev/null @@ -1,500 +0,0 @@ -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 0 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 0 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 0 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 2 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 4 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 5 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 5 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 5 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 8 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 9 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 10 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 11 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 12 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 12 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 15 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 15 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 17 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 18 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 18 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 19 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 20 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 24 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 24 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 26 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 26 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 27 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 28 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 30 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 33 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 34 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 35 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 35 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 35 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 37 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 37 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 41 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 42 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 42 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 43 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 44 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 47 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 51 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 51 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 53 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 54 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 57 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 58 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 58 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 64 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 65 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 66 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 67 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 67 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 69 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 70 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 70 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 70 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 72 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 72 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 74 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 76 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 76 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 77 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 78 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 80 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 82 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 83 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 83 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 84 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 84 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 85 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 86 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 87 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 90 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 90 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 90 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 92 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 95 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 95 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 96 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 97 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 97 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 98 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 98 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 100 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 100 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 103 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 103 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 104 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 104 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 105 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 111 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 113 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 113 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 114 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 116 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 118 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 118 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 119 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 119 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 119 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 120 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 120 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 125 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 125 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 126 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 128 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 128 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 128 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 129 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 129 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 131 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 133 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 134 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 134 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 136 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 137 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 137 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 138 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 138 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 138 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 138 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 143 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 145 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 146 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 146 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 149 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 149 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 150 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 152 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 152 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 153 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 155 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 156 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 157 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 158 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 160 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 162 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 163 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 164 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 164 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 165 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 165 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 166 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 167 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 167 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 167 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 168 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 169 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 169 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 169 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 169 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 170 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 172 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 172 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 174 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 174 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 175 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 175 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 176 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 176 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 177 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 178 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 179 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 179 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 180 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 181 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 183 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 186 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 187 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 187 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 187 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 189 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 190 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 191 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 191 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 192 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 193 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 193 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 193 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 194 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 195 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 195 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 196 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 197 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 197 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 199 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 199 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 199 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 200 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 200 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 201 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 202 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 203 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 203 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 205 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 205 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 207 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 207 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 208 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 208 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 208 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 209 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 209 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 213 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 213 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 214 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 216 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 216 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 217 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 217 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 218 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 219 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 219 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 221 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 221 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 222 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 223 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 223 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 224 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 224 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 226 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 228 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 229 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 229 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 233 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 233 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 235 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 237 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 237 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 238 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 238 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 239 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 239 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 241 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 242 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 242 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 244 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 247 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 248 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 249 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 252 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 255 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 255 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 256 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 256 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 257 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 258 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 260 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 262 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 263 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 265 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 265 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 266 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 272 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 272 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 273 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 273 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 273 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 274 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 275 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 277 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 277 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 277 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 277 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 278 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 278 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 280 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 280 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 281 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 281 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 282 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 282 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 283 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 284 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 285 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 286 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 287 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 288 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 288 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 289 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 291 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 292 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 296 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 298 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 298 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 298 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 302 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 305 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 306 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 307 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 307 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 308 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 309 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 309 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 310 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 311 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 311 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 311 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 315 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 316 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 316 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 316 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 317 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 317 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 318 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 318 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 318 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 321 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 321 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 322 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 322 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 323 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 325 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 325 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 327 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 327 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 327 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 331 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 331 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 332 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 333 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 333 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 335 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 336 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 338 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 339 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 341 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 342 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 342 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 344 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 344 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 345 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 351 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 353 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 353 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 356 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 360 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 362 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 364 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 365 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 366 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 367 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 367 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 368 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 369 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 369 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 369 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 373 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 374 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 375 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 377 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 378 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 379 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 382 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 382 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 384 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 384 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 384 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 386 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 389 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 392 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 393 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 394 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 395 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 395 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 396 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 396 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 396 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 397 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 397 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 399 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 399 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 400 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 402 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 403 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 403 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 403 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 404 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 404 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 406 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 406 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 406 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 406 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 407 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 409 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 409 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 409 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 411 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 413 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 413 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 414 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 414 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 417 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 417 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 417 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 418 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 419 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 421 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 424 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 424 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 427 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 429 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 429 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 430 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 430 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 430 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 431 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 431 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 431 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 432 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 435 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 436 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 437 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 438 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 438 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 438 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 439 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 439 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 443 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 444 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 446 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 448 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 449 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 452 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 453 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 454 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 454 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 454 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 455 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 457 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 458 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 458 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 459 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 459 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 460 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 462 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 462 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 463 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 463 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 466 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 466 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 466 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 467 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 468 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 468 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 468 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 468 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 470 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 472 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 475 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 477 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 478 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 478 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 479 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 480 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 480 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 480 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 481 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 482 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 483 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 484 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 485 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 487 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 489 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 489 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 489 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 489 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 490 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 491 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 492 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 492 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 493 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 494 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 495 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 496 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 497 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 498 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 498 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 498 2449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 b/sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index 9fdb526d945e0..5bc08062d30eb 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -42,6 +42,12 @@ log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF +log4j.additivity.hive.log=false +log4j.logger.hive.log=OFF + +log4j.additivity.parquet.hadoop.ParquetRecordReader=false +log4j.logger.parquet.hadoop.ParquetRecordReader=OFF + log4j.additivity.hive.ql.metadata.Hive=false log4j.logger.hive.ql.metadata.Hive=OFF 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 new file mode 100644 index 0000000000000..c640ca148b70b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q @@ -0,0 +1,43 @@ + +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 new file mode 100644 index 0000000000000..51f231008f6d2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv @@ -0,0 +1 @@ +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 new file mode 100644 index 0000000000000..18e02dc854baf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv @@ -0,0 +1 @@ +SET hive.vectorized.execution.enabled = true; 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 index 4881757a4613a..4193315d30043 100644 --- 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 @@ -1,9 +1,9 @@ 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; +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`; 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 index 5f9d5ef9ca09e..8cbb25cfa9725 100644 --- 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 @@ -2,7 +2,7 @@ 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'); +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 index 92af30b6b5a15..3c0ff02b1ac1d 100644 --- 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 @@ -2,8 +2,8 @@ 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'); +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 index 135411fd320ef..f2135b1aa02e4 100644 --- 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 @@ -2,8 +2,8 @@ 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'); +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 index 899145deaf8c5..7376d8bfe4a74 100644 --- 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 @@ -2,8 +2,8 @@ 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'); +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'; 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 index 26ba2878903fe..be971f1849869 100644 --- 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 @@ -1,5 +1,5 @@ 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 ; +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 ; 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 index 6e51c2f7629c9..4babdda2dbe2a 100644 --- 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 @@ -1,5 +1,5 @@ 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 ; +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 ; 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 index 2d4ce0b9f6051..3af807ef61217 100644 --- 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 @@ -1,5 +1,5 @@ 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 ; +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 ; 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 index fdf20f850e8f8..9e8bcbd1bbf78 100644 --- 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 @@ -1,2 +1,3 @@ +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 index de59bc579a0fe..33d4aed3cd9ad 100644 --- 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 @@ -1,2 +1,3 @@ +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_non_existent_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q new file mode 100644 index 0000000000000..78a97019f192e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q @@ -0,0 +1 @@ +analyze table nonexistent compute statistics; 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 index bea25391628a6..130b37b5c9d52 100644 --- 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 @@ -14,5 +14,5 @@ create table tstsrcpart like srcpart; -- 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'); +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/authorization_addjar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q new file mode 100644 index 0000000000000..a1709dae5f5b8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q @@ -0,0 +1,7 @@ +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 new file mode 100644 index 0000000000000..8abdd2b3cde8b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..f716262e23bbb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..f9049350180ee --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q @@ -0,0 +1,8 @@ +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 new file mode 100644 index 0000000000000..de91e91923308 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..42a42f65b28a9 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..0d14cde6d5460 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..d5ea284f14749 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q @@ -0,0 +1,17 @@ +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 new file mode 100644 index 0000000000000..02bbe090cfba7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q @@ -0,0 +1,7 @@ +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 new file mode 100644 index 0000000000000..8760fa8d82259 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q @@ -0,0 +1,8 @@ +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 new file mode 100644 index 0000000000000..c904a100c515c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q @@ -0,0 +1,8 @@ +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 new file mode 100644 index 0000000000000..a84fe64bd618e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..9b1f2ea6c6acb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..1cf74a365d79e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..47663c9bb93e3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q @@ -0,0 +1,14 @@ +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 new file mode 100644 index 0000000000000..7d47a7b64967b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q @@ -0,0 +1,7 @@ +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 new file mode 100644 index 0000000000000..64b300c8d9b2f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..edeae9b71d7ac --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q @@ -0,0 +1,22 @@ +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 new file mode 100644 index 0000000000000..46d4d0f92c8e3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q @@ -0,0 +1,27 @@ +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 new file mode 100644 index 0000000000000..a7aa17f5abfcf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..f05e9458fa804 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q @@ -0,0 +1,11 @@ +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_create_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q new file mode 100644 index 0000000000000..d969e39027e99 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q @@ -0,0 +1,5 @@ +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 new file mode 100644 index 0000000000000..87719b0043e2e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q @@ -0,0 +1,5 @@ +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 new file mode 100644 index 0000000000000..f3c86b97ce76f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q @@ -0,0 +1,14 @@ +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 new file mode 100644 index 0000000000000..7808cb3ec7b39 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q @@ -0,0 +1,16 @@ +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 new file mode 100644 index 0000000000000..8dc8e45a79075 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..d51c1c3507eef --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q @@ -0,0 +1,14 @@ +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 new file mode 100644 index 0000000000000..2fa3cb260b07e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..b9bee4ea40d40 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..2a1da23daeb18 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..9c724085d9901 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q @@ -0,0 +1,5 @@ +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 new file mode 100644 index 0000000000000..8a3300cb2e378 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..0172c4c74c82d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..2d0e52da008d8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..76bbab42b3750 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q @@ -0,0 +1,11 @@ +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_priv_current_role_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q new file mode 100644 index 0000000000000..bbf3b66970b6a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q @@ -0,0 +1,29 @@ +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 new file mode 100644 index 0000000000000..002389f203e25 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q @@ -0,0 +1 @@ +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 new file mode 100644 index 0000000000000..69c5a8de8b05f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q @@ -0,0 +1 @@ +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 new file mode 100644 index 0000000000000..e19bf370fa077 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q @@ -0,0 +1,14 @@ +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 new file mode 100644 index 0000000000000..4b0cf3286ae71 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q @@ -0,0 +1,18 @@ +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 new file mode 100644 index 0000000000000..a819d204f56b6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q @@ -0,0 +1,12 @@ +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 new file mode 100644 index 0000000000000..423f030630b6c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q @@ -0,0 +1,24 @@ +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 new file mode 100644 index 0000000000000..c5c500a71251f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q @@ -0,0 +1,22 @@ +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 new file mode 100644 index 0000000000000..d9f4c7cdb850b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q @@ -0,0 +1,74 @@ +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 new file mode 100644 index 0000000000000..39871793af398 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q @@ -0,0 +1,9 @@ +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 new file mode 100644 index 0000000000000..a4071cd0d4d87 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..9ba3a82a5608e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..03f748fcc9b7a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q @@ -0,0 +1,16 @@ +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 new file mode 100644 index 0000000000000..d8190de950de7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..2afe87fc30c9e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..69cea2f2673f0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q @@ -0,0 +1,2 @@ +-- 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 new file mode 100644 index 0000000000000..0fc9fca940c39 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..285600b23a149 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q @@ -0,0 +1,9 @@ +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 new file mode 100644 index 0000000000000..d82ac710cc3ba --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..d38ba74d9006a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q @@ -0,0 +1,16 @@ +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 new file mode 100644 index 0000000000000..c446b8636fb32 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q @@ -0,0 +1,13 @@ +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 new file mode 100644 index 0000000000000..c8e1fb43ee317 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..c8549b4563b2f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..edfdf5a8fc407 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q @@ -0,0 +1,12 @@ +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 new file mode 100644 index 0000000000000..81763916a0b81 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q @@ -0,0 +1,22 @@ +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 new file mode 100644 index 0000000000000..4ea4dc0a4747a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q @@ -0,0 +1,25 @@ +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 new file mode 100644 index 0000000000000..1a8f9cb2ad197 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q @@ -0,0 +1,13 @@ +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 new file mode 100644 index 0000000000000..81b6e522c1abb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q @@ -0,0 +1,14 @@ +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 new file mode 100644 index 0000000000000..0a2fd8919f455 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q @@ -0,0 +1,14 @@ +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 new file mode 100644 index 0000000000000..6af41f0cdaa23 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q @@ -0,0 +1,11 @@ +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 new file mode 100644 index 0000000000000..d8beac370d4b6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q @@ -0,0 +1,10 @@ +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 new file mode 100644 index 0000000000000..bfd3165237774 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q @@ -0,0 +1 @@ +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 new file mode 100644 index 0000000000000..2b29822371b19 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q @@ -0,0 +1 @@ +revoke role PUBLIC from user hive_test_user; 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 index 048a8fd5cfb8e..6bebb8942d613 100644 --- 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 @@ -1,19 +1,19 @@ 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' +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' +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' +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' +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' +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. 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 index 9478a2f1b989c..802fcd903c0ac 100644 --- 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 @@ -8,10 +8,10 @@ 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 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; +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; 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 index 69afe0ae700ed..ac5abebb0b4b6 100644 --- 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 @@ -8,13 +8,13 @@ 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='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 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; +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; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q deleted file mode 100644 index 0ff44779657ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN -SELECT x.key, x.value as key FROM SRC x CLUSTER BY key; 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 index af923504c8c4f..b4887c4115854 100644 --- 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 @@ -3,12 +3,12 @@ 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'); +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 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 index d9725ddc7fdf6..2f8e9271ddd34 100644 --- 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 @@ -3,12 +3,12 @@ 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'); +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 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 index eb73962ba133c..34f91fc8d1de8 100644 --- 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 @@ -3,12 +3,12 @@ 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'); +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 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 index dbfaaecbdf614..49d89dd12132b 100644 --- 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 @@ -3,12 +3,12 @@ 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'); +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 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 index ca8548958fcd6..a4e0056bff370 100644 --- 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 @@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none ( 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; +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; 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 index 5bbd70d86b297..85a5f0a021940 100644 --- 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 @@ -8,7 +8,7 @@ CREATE TABLE table_complex_type ( d MAP> ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type; +LOAD DATA LOCAL INPATH '../../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type; explain 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 index ca8548958fcd6..a4e0056bff370 100644 --- 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 @@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none ( 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; +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; 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 new file mode 100644 index 0000000000000..c314a940f95c2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q @@ -0,0 +1,8 @@ + +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 new file mode 100644 index 0000000000000..5974811280350 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q @@ -0,0 +1,7 @@ +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 new file mode 100644 index 0000000000000..3b71e00b2eaa2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q @@ -0,0 +1 @@ +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 new file mode 100644 index 0000000000000..ae95391edd3e5 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q @@ -0,0 +1 @@ +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 new file mode 100644 index 0000000000000..2083064593299 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q @@ -0,0 +1 @@ +create function default.badfunc as 'java.lang.String'; 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 new file mode 100644 index 0000000000000..2160b4719662b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q @@ -0,0 +1,4 @@ +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 new file mode 100644 index 0000000000000..e52a1d97db801 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q @@ -0,0 +1 @@ +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/date_literal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q deleted file mode 100644 index b7fac0d3df346..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q +++ /dev/null @@ -1,2 +0,0 @@ --- Not in YYYY-MM-DD format -SELECT DATE '2001-1-1' FROM src LIMIT 2; 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 new file mode 100644 index 0000000000000..1c658c79b99ea --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..ef4b323f063b6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..4a0c6c25c67c7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..0b00046579f43 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q @@ -0,0 +1,6 @@ +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/deletejar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q index 7b0c92311a11a..0bd6985e031b6 100644 --- 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 @@ -1,4 +1,4 @@ -ADD JAR ../data/files/TestSerDe.jar; -DELETE JAR ../data/files/TestSerDe.jar; +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/drop_func_nonexistent.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q new file mode 100644 index 0000000000000..892ef00e3f86c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q @@ -0,0 +1,3 @@ +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_partition_filter_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q deleted file mode 100644 index 4d238d73a9116..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q +++ /dev/null @@ -1,11 +0,0 @@ -create table ptestfilter (a string, b int) partitioned by (c string, d int); -describe ptestfilter; - -alter table ptestfilter add partition (c='US', d=1); -alter table ptestfilter add partition (c='US', d=2); -show partitions ptestfilter; - -alter table ptestfilter drop partition (c='US', d<'2'); - - - 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 index 0be2e71c94dff..0ad99d100dc07 100644 --- 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 @@ -8,7 +8,7 @@ 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); +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 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 index e4f0daca92bd1..ca60d047efdd5 100644 --- 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 @@ -3,8 +3,8 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='h1'); -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='h2'); +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; 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 index 4d1e0a62a431f..7083edc32b98d 100644 --- 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 @@ -8,5 +8,5 @@ ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05'); 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 +-- 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 index 23777db3ea93d..6dfe81a8b0568 100644 --- 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 @@ -9,5 +9,5 @@ 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_test2 table partition (ds='2013-04-05', hr='3') already exists thus this query will fail +-- 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 index 350bf248acc9c..60671e52e05d5 100644 --- 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 @@ -9,5 +9,5 @@ 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', hr='1') already exists thus this query will fail +-- 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 index 81944b3330853..38c0eda2368bd 100644 --- 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 @@ -2,5 +2,5 @@ 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_test1 partition (ds='2013-04-05') does not exist thus this query will fail +-- 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/exim_00_unsupported_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q index d86ecd5785d02..6ffc33acb92ec 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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 index 5f3223152f766..970e6463e24a5 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -16,9 +16,9 @@ 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; +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 ../build/ql/test/data/exports/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 index d7204dc478d25..358918363d830 100644 --- 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 @@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -27,12 +27,12 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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 ../build/ql/test/data/exports/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 index 6cd7eda455ee6..45268c21c00e3 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -18,6 +18,6 @@ create table exim_department ( dep_key int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index 7f3f577c433b3..cad6c90fd316e 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -18,6 +18,6 @@ create table exim_department ( dep_id int comment "department id", dep_name stri tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index d3ec9fff82bf9..f5f904f42af5a 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -18,6 +18,6 @@ create table exim_department ( dep_id bigint comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index 1cc691fc2912d..c56329c03f89f 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -18,6 +18,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index 27830ad5f93dd..afaedcd37bf72 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -21,6 +21,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index d85048a97a876..230b28c402cc5 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -19,6 +19,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index 84b3786a161c6..c2e00a9663468 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -23,6 +23,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index eaf9c579d51d4..a6586ead0c23f 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -19,6 +19,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index 092fd779541c3..990a686ebeea6 100644 --- 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 @@ -5,9 +5,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -20,6 +20,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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_13_nonnative_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q index 05de3d77b07b8..02537ef022d82 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -18,7 +18,7 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index dc194ca814ee4..897c6747354b7 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -19,7 +19,7 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index e233707cc4db2..12013e5ccfc49 100644 --- 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 @@ -5,9 +5,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -19,7 +19,7 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index a10788e3e3b6b..d8d2b8008c9ee 100644 --- 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 @@ -5,9 +5,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -20,7 +20,7 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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 index cc4a56ca34b88..82dcce9455958 100644 --- 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 @@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -25,6 +25,6 @@ import table exim_employee partition (emp_country="us") from 'ql/test/data/expor describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/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 index 140e3bb3b1bbc..d92efeb9a70ef 100644 --- 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 @@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -25,6 +25,6 @@ import table exim_employee partition (emp_country="us", emp_state="kl") from 'ql describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/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 index 048befe4d3d43..12d827b9c838c 100644 --- 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 @@ -4,9 +4,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -17,7 +17,7 @@ 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 ../build/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 index 89cbb9ecd8086..726dee53955af 100644 --- 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 @@ -4,17 +4,17 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +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 @@ -22,9 +22,9 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import table exim_department from 'ql/test/data/exports/exim_department' location 'ql/test/data/tablestore2/exim_department'; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop table exim_department; -dfs -rmr ../build/ql/test/data/tablestore/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 index 0cbfc85258d2c..d187c78202034 100644 --- 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 @@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -29,7 +29,7 @@ create table exim_employee ( emp_id int comment "employee id") tblproperties("creator"="krishna"); import external table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee'; -dfs -rmr ../build/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 index d9ab0cf0e4e50..b818686f773df 100644 --- 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 @@ -5,8 +5,8 @@ create table exim_department ( dep_id int) stored as textfile; set hive.security.authorization.enabled=true; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/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'; set hive.security.authorization.enabled=false; 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 index 2dbd534074fa8..4acefb9f0ae12 100644 --- 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 @@ -2,9 +2,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -18,5 +18,5 @@ import from 'ql/test/data/exports/exim_department'; set hive.security.authorization.enabled=false; drop table exim_department; drop database importer; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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 index ccbcee3698dae..467014e4679f6 100644 --- 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 @@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id") 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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -26,6 +26,6 @@ set hive.security.authorization.enabled=true; import from 'ql/test/data/exports/exim_employee'; set hive.security.authorization.enabled=false; -dfs -rmr ../build/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_25_import_nonexist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q index 50bfe005c4278..595fa7e764952 100644 --- 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 @@ -3,9 +3,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -19,5 +19,5 @@ set hive.security.authorization.enabled=false; select * from exim_department; drop table exim_department; drop database importer; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; 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 index 9f44f225e955a..82230f782eac3 100644 --- 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 @@ -2,6 +2,6 @@ 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; +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 new file mode 100644 index 0000000000000..286cf1afb491b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q @@ -0,0 +1,13 @@ +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/illegal_partition_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q index 1cdaffd1f31a3..1ab828c8beae4 100644 --- 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 @@ -1,6 +1,6 @@ -- 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'); +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_type3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q new file mode 100644 index 0000000000000..49e6a092fc127 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q @@ -0,0 +1,4 @@ +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 new file mode 100644 index 0000000000000..50f486e6245cf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q @@ -0,0 +1,3 @@ +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_compact_entry_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q index 7d003e3e4b440..5bb889c027743 100644 --- 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 @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; drop index src_index on src; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; 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 index d79674539a1b2..c6600e69b6a7c 100644 --- 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 @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; drop index src_index on src; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; 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 new file mode 100644 index 0000000000000..c20c168a887c2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q @@ -0,0 +1,9 @@ +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 new file mode 100644 index 0000000000000..a92ee5ca94a33 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q @@ -0,0 +1,17 @@ +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/insertover_dynapart_ifnotexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q index cbf65c4ac69fe..a8f77c28a8251 100644 --- 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 @@ -4,6 +4,6 @@ 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); +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_char_length_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q new file mode 100644 index 0000000000000..ba7d164c77155 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q @@ -0,0 +1,2 @@ +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 new file mode 100644 index 0000000000000..866b43d31273d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q @@ -0,0 +1 @@ +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 new file mode 100644 index 0000000000000..481b630d20489 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q @@ -0,0 +1,3 @@ +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_columns.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q deleted file mode 100644 index 14b3409cb4cc4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q +++ /dev/null @@ -1,4 +0,0 @@ -ADD JAR ../data/files/TestSerDe.jar; -CREATE TABLE DELETEJAR(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' -STORED AS TEXTFILE -TBLPROPERTIES('columns'='valid_colname,invalid.colname'); 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 new file mode 100644 index 0000000000000..e39a38e2fcd47 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..c0da913c28812 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q @@ -0,0 +1,6 @@ + + +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 new file mode 100644 index 0000000000000..8e219637eb0c6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q @@ -0,0 +1,5 @@ + +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/limit_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q new file mode 100644 index 0000000000000..d59394544ccf0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q @@ -0,0 +1,7 @@ +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 new file mode 100644 index 0000000000000..0afd4a965ab94 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q @@ -0,0 +1,18 @@ +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/load_exist_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q index f86cd92d9d4c7..eb72d940a5392 100644 --- 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 @@ -1,4 +1,4 @@ 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'); +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 index 387aaed9a1e50..75a5216e00d82 100644 --- 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 @@ -2,4 +2,4 @@ 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; +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 index d807c698777f1..32653631ad6a4 100644 --- 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 @@ -1,3 +1,3 @@ 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 ; +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 index c409d5a94a9ce..315988dc0a959 100644 --- 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 @@ -1,3 +1,3 @@ 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'); +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 index 356c16a664386..81517991b26fa 100644 --- 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 @@ -1,2 +1,2 @@ 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; +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 index eed5651cbf20b..c56f0d408d4ad 100644 --- 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 @@ -4,4 +4,4 @@ set hive.mapred.supports.subdirectories=true; 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; +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 index 927f02e82bf91..64182eac8362d 100644 --- 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 @@ -1,3 +1,3 @@ DROP VIEW xxx11; CREATE VIEW xxx11 AS SELECT * FROM src; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE xxx11; +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 index 16feeca22649a..f0c3b59d30ddf 100644 --- 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 @@ -3,4 +3,4 @@ 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; +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 index 7e589fbfde620..4d79bbeb102c7 100644 --- 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 @@ -3,4 +3,4 @@ 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 +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 index ff5ed4e2e3107..050c819a2f04b 100644 --- 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 @@ -3,4 +3,4 @@ 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 +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 index ffb64ed643b14..7f5ad754142ab 100644 --- 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 @@ -1,3 +1,3 @@ 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'); +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 index 8f4b37a9d49f6..ed9e21dd8a1fb 100644 --- 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 @@ -1,4 +1,4 @@ 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); +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/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 new file mode 100644 index 0000000000000..4966f2b9b2825 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q @@ -0,0 +1,17 @@ +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 new file mode 100644 index 0000000000000..1f9ad90898dce --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q @@ -0,0 +1,6 @@ +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 new file mode 100644 index 0000000000000..8cbe31083b400 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q @@ -0,0 +1,8 @@ +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 new file mode 100644 index 0000000000000..4127a6f150a13 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q @@ -0,0 +1,15 @@ +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/nested_complex_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q index ac6c4ee549d85..09f13f52aeadf 100644 --- 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 @@ -10,6 +10,6 @@ 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; +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/nopart_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q index 4841f9e11c84e..6669bf62d8822 100644 --- 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 @@ -2,6 +2,6 @@ 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) +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 index 6e5ad6eb41a8b..966982fd5ce52 100644 --- 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 @@ -1,5 +1,5 @@ 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 ; +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_alias3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q deleted file mode 100644 index 6cc3e87288a89..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE; - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.key, sum(src.value) WHERE src.key < 100 group by key; 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 new file mode 100644 index 0000000000000..745a7867264e3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..89d3602fd3e97 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..8a4973110a51f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..4ef36fa0efc49 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q @@ -0,0 +1,3 @@ +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 new file mode 100644 index 0000000000000..55825f76dc240 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q @@ -0,0 +1,3 @@ +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/protectmode_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q index 72b55ea25d8e9..3fdc036996563 100644 --- 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 @@ -4,6 +4,6 @@ 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'); +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'); +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/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q index 542367ace22e0..ef372259ed3e3 100644 --- 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 @@ -12,7 +12,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- testAggrFuncsWithNoGBYNoPartDef select p_mfgr, 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 index 95b35113e3cf0..58430423436b7 100644 --- 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 @@ -12,7 +12,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- testAmbiguousWindowDefn select p_mfgr, p_name, p_size, 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 index 8333ddc948419..a171961a683ee 100644 --- 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 @@ -1,2 +1,3 @@ +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 index d1aa1f1a9542d..7bac1c775522b 100644 --- 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 @@ -1,2 +1,3 @@ +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 index 53971916e6c96..300d145508887 100644 --- 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 @@ -1,2 +1,3 @@ +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/script_broken_pipe1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q deleted file mode 100644 index 6b1c09decfb35..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q +++ /dev/null @@ -1,3 +0,0 @@ -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/script_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q index e46aed03b147f..8ca849b82d8ad 100644 --- 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 @@ -1,7 +1,7 @@ EXPLAIN -SELECT TRANSFORM(src.key, src.value) USING '../data/scripts/error_script' AS (tkey, tvalue) +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) +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/serde_regex2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q index a3955744221a4..d523d03e906c1 100644 --- 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 @@ -16,8 +16,8 @@ WITH SERDEPROPERTIES ( ) 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; +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/set_hiveconf_validation2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q new file mode 100644 index 0000000000000..579e9408b6c35 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q @@ -0,0 +1,5 @@ +-- 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/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q index 401cc37f67dd0..1b2872d3d7ed8 100644 --- 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 @@ -6,7 +6,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +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; 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 index c7e63591adde0..0fa9ff6820371 100644 --- 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 @@ -5,7 +5,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; set hive.stats.default.aggregator=""; 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 index 7fa0f55f2a450..be7c4f72feb9b 100644 --- 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 @@ -6,7 +6,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +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; 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 index f82d4b54b697b..652afe7c5bfba 100644 --- 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 @@ -5,7 +5,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=""; set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; 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 new file mode 100644 index 0000000000000..9013df6f938dc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q @@ -0,0 +1,10 @@ + + +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 new file mode 100644 index 0000000000000..a9bc6ee6a38cb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q @@ -0,0 +1,5 @@ + + +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 new file mode 100644 index 0000000000000..1365389cb269a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q @@ -0,0 +1,6 @@ + + + +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 new file mode 100644 index 0000000000000..6805c5b16b0f4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q @@ -0,0 +1,7 @@ + + +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 new file mode 100644 index 0000000000000..e8c41e6b17ae7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q @@ -0,0 +1,18 @@ + + +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 new file mode 100644 index 0000000000000..852b2953ff463 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q @@ -0,0 +1,10 @@ + + +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 new file mode 100644 index 0000000000000..d442f077c0707 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q @@ -0,0 +1,6 @@ + + +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 new file mode 100644 index 0000000000000..8ea94c5fc6d76 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q @@ -0,0 +1,6 @@ + +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 new file mode 100644 index 0000000000000..99ff9ca703835 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q @@ -0,0 +1,6 @@ + + +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 new file mode 100644 index 0000000000000..105d3d22d9d2b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q @@ -0,0 +1,26 @@ +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 new file mode 100644 index 0000000000000..c2c322178f386 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q @@ -0,0 +1,5 @@ + +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/udf_case_type_wrong.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q deleted file mode 100644 index 2fb5ff74cc3ce..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE '1' - WHEN 1 THEN 2 - WHEN 3 THEN 4 - ELSE 5 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q deleted file mode 100644 index 5772dc1a95c9d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE 1 - WHEN 1 THEN '2' - WHEN 3 THEN 4 - ELSE 5 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q deleted file mode 100644 index 5aaf0188eb9c6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE 1 - WHEN 1 THEN NULL - WHEN 3 THEN '2' - ELSE 7 - END -FROM src LIMIT 1; 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 new file mode 100644 index 0000000000000..68050fd95cd22 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q @@ -0,0 +1 @@ +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 new file mode 100644 index 0000000000000..bcfa217737e33 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q @@ -0,0 +1 @@ +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_nonexistent_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q new file mode 100644 index 0000000000000..d37665dde69bc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q @@ -0,0 +1 @@ +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_qualified_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q new file mode 100644 index 0000000000000..476dfa21a237f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q @@ -0,0 +1 @@ +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_when_type_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q deleted file mode 100644 index 79fa65f63da56..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE - WHEN 1=2 THEN '2' - WHEN 3=4 THEN 4 - ELSE 5 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q deleted file mode 100644 index 8bb5fdd7ea377..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE - WHEN 1=2 THEN '2' - WHEN 3=4 THEN '5' - ELSE 5.3 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q deleted file mode 100644 index 3c4204f780004..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q +++ /dev/null @@ -1,6 +0,0 @@ - -CREATE TEMPORARY FUNCTION example_arraysum AS 'org.apache.hadoop.hive.contrib.udf.example.UDFExampleArraySum'; - -SELECT example_arraysum(lint)FROM src_thrift; - -DROP TEMPORARY FUNCTION example_arraysum; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q deleted file mode 100644 index e3c5c830897ea..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q +++ /dev/null @@ -1,4 +0,0 @@ -explain -select s1.key as key, s1.value as value from src s1 - UNION ALL -select s2.key as key, s2.value as value from src s2; 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 new file mode 100644 index 0000000000000..c5b593e4bb556 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q @@ -0,0 +1 @@ +select nonexistfunc(key) over () from src limit 1; 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 new file mode 100644 index 0000000000000..15f8fae292bbb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q @@ -0,0 +1,26 @@ +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/clientpositive/alter1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q index 5fd19455f93cd..312a0177fafff 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q @@ -15,7 +15,7 @@ describe extended alter1; alter table alter1 set serdeproperties('s1'='10', 's2' ='20'); describe extended alter1; -add jar ../data/files/TestSerDe.jar; +add jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; alter table alter1 set serde 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties('s1'='9'); describe extended alter1; @@ -56,7 +56,7 @@ DESCRIBE EXTENDED alter1; ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='10', 's2' ='20'); DESCRIBE EXTENDED alter1; -add jar ../data/files/TestSerDe.jar; +add jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' WITH SERDEPROPERTIES ('s1'='9'); DESCRIBE EXTENDED alter1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q index 3cde00748b98e..91e4e9bad0eda 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q @@ -1,5 +1,5 @@ create table alter3_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter3_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter3_src ; create table alter3 ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; @@ -34,7 +34,7 @@ USE alter3_db; SHOW TABLES; CREATE TABLE alter3_src (col1 STRING) STORED AS TEXTFILE ; -LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE alter3_src ; +LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE alter3_src ; CREATE TABLE alter3 (col1 STRING) PARTITIONED BY (pcol1 STRING, pcol2 STRING) STORED AS SEQUENCEFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q index 0d14f228d1c30..66c9f8dc54b91 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q @@ -3,7 +3,7 @@ -- create table alter5_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter5_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter5_src ; create table alter5 ( col1 string ) partitioned by (dt string); @@ -32,7 +32,7 @@ USE alter5_db; SHOW TABLES; create table alter5_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter5_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter5_src ; create table alter5 ( col1 string ) partitioned by (dt string); alter table alter5 add partition (dt='a') location 'parta'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q new file mode 100644 index 0000000000000..4ecb7e7389f6b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q @@ -0,0 +1,32 @@ +drop table alter_char_1; + +create table alter_char_1 (key string, value string); +insert overwrite table alter_char_1 + select key, value from src order by key limit 5; + +select * from alter_char_1 order by key; + +-- change column to char +alter table alter_char_1 change column value value char(20); +-- contents should still look the same +select * from alter_char_1 order by key; + +-- change column to smaller char +alter table alter_char_1 change column value value char(3); +-- value column should be truncated now +select * from alter_char_1 order by key; + +-- change back to bigger char +alter table alter_char_1 change column value value char(20); +-- column values should be full size again +select * from alter_char_1 order by key; + +-- add char column +alter table alter_char_1 add columns (key2 int, value2 char(10)); +select * from alter_char_1 order by key; + +insert overwrite table alter_char_1 + select key, value, key, value from src order by key limit 5; +select * from alter_char_1 order by key; + +drop table alter_char_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q new file mode 100644 index 0000000000000..7fa9fcef11a14 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q @@ -0,0 +1,22 @@ + +-- alter column type, with partitioned table +drop table if exists alter_char2; + +create table alter_char2 ( + c1 char(255) +) partitioned by (hr int); + +insert overwrite table alter_char2 partition (hr=1) + select value from src limit 1; + +select c1, length(c1) from alter_char2; + +alter table alter_char2 change column c1 c1 char(10); + +select hr, c1, length(c1) from alter_char2 where hr = 1; + +insert overwrite table alter_char2 partition (hr=2) + select key from src limit 1; + +select hr, c1, length(c1) from alter_char2 where hr = 1; +select hr, c1, length(c1) from alter_char2 where hr = 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q index 807ef539c8643..e1c3780a9b0ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q @@ -1,9 +1,9 @@ set hive.exec.concatenate.check.index =false; 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; +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`; @@ -26,9 +26,9 @@ create table src_rc_concatenate_test_part(key int, value string) partitioned by alter table src_rc_concatenate_test_part add partition (ds='2011'); -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_concatenate_test_part partition (ds='2011'); -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_concatenate_test_part partition (ds='2011'); -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_concatenate_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test_part partition (ds='2011'); show table extended like `src_rc_concatenate_test_part` partition (ds='2011'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q new file mode 100644 index 0000000000000..b224f3339e4ab --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q @@ -0,0 +1,9 @@ + +create database db_alter_onr; +describe database db_alter_onr; + +alter database db_alter_onr set owner user user1; +describe database db_alter_onr; + +alter database db_alter_onr set owner role role1; +describe database db_alter_onr; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q index c3502739d54a9..ceabd0830a03d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q @@ -1,8 +1,8 @@ create table src_rc_merge_test(key int, value string) stored as rcfile; -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test; -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test; -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test; +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test; +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test; +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test; show table extended like `src_rc_merge_test`; @@ -21,9 +21,9 @@ create table src_rc_merge_test_part(key int, value string) partitioned by (ds st alter table src_rc_merge_test_part add partition (ds='2011'); -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2011'); -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2011'); -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2011'); show table extended like `src_rc_merge_test_part` partition (ds='2011'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q index 65ddfed13dac4..e09703d1c6c85 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q @@ -3,9 +3,9 @@ create table src_rc_merge_test_part(key int, value string) partitioned by (ds st alter table src_rc_merge_test_part add partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); desc extended src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); select count(1) from src_rc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31'; select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q index 23bae55ab2add..0af87e2c076e6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q @@ -1,8 +1,8 @@ create table src_rc_merge_test_stat(key int, value string) stored as rcfile; -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat; -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_stat; -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_stat; +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat; +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_stat; +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_stat; show table extended like `src_rc_merge_test_stat`; desc extended src_rc_merge_test_stat; @@ -21,9 +21,9 @@ create table src_rc_merge_test_part_stat(key int, value string) partitioned by ( alter table src_rc_merge_test_part_stat add partition (ds='2011'); -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); show table extended like `src_rc_merge_test_part_stat` partition (ds='2011'); desc extended src_rc_merge_test_part_stat; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q index b6d1eb8f2d4b8..5dda4c08fd5b3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q @@ -1,4 +1,4 @@ - +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) create table tst1(key string, value string) partitioned by (ds string) clustered by (key) into 10 buckets; alter table tst1 clustered by (key) into 8 buckets; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q index c6a4ad24fcdde..acc028bdd81eb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q @@ -1,3 +1,4 @@ +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) -- Tests that when overwriting a partition in a table after altering the bucketing/sorting metadata -- the partition metadata is updated as well. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q new file mode 100644 index 0000000000000..d81430441c720 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q @@ -0,0 +1,85 @@ +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) +-- Tests that when overwriting a partition in a table after altering the bucketing/sorting metadata +-- the partition metadata is updated as well. + +CREATE TABLE tst1(key STRING, value STRING) PARTITIONED BY (ds STRING); + +DESCRIBE FORMATTED tst1; + +SET hive.enforce.bucketing=true; +SET hive.enforce.sorting=true; +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test an unbucketed partition gets converted to bucketed +ALTER TABLE tst1 CLUSTERED BY (key) INTO 8 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test an unsorted partition gets converted to sorted +ALTER TABLE tst1 CLUSTERED BY (key) SORTED BY (key DESC) INTO 8 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test changing the bucket columns +ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (key DESC) INTO 8 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test changing the number of buckets +ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (key DESC) INTO 4 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test changing the sort columns +ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (value DESC) INTO 4 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test changing the sort order +ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (value ASC) INTO 4 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test a sorted partition gets converted to unsorted +ALTER TABLE tst1 CLUSTERED BY (value) INTO 4 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test a bucketed partition gets converted to unbucketed +ALTER TABLE tst1 NOT CLUSTERED; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q new file mode 100644 index 0000000000000..a03992510b072 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q @@ -0,0 +1,59 @@ +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) +create table tst1(key string, value string) partitioned by (ds string) clustered by (key) into 10 buckets; + +alter table tst1 clustered by (key) into 8 buckets; + +describe formatted tst1; + +set hive.enforce.bucketing=true; +insert overwrite table tst1 partition (ds='1') select key, value from src; + +describe formatted tst1 partition (ds = '1'); + +-- Test changing bucket number + +alter table tst1 clustered by (key) into 12 buckets; + +insert overwrite table tst1 partition (ds='1') select key, value from src; + +describe formatted tst1 partition (ds = '1'); + +describe formatted tst1; + +-- Test changing bucket number of (table/partition) + +alter table tst1 into 4 buckets; + +describe formatted tst1; + +describe formatted tst1 partition (ds = '1'); + +alter table tst1 partition (ds = '1') into 6 buckets; + +describe formatted tst1; + +describe formatted tst1 partition (ds = '1'); + +-- Test adding sort order + +alter table tst1 clustered by (key) sorted by (key asc) into 12 buckets; + +describe formatted tst1; + +-- Test changing sort order + +alter table tst1 clustered by (key) sorted by (value desc) into 12 buckets; + +describe formatted tst1; + +-- Test removing test order + +alter table tst1 clustered by (value) into 12 buckets; + +describe formatted tst1; + +-- Test removing buckets + +alter table tst1 not clustered; + +describe formatted tst1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q index 5479afbbd5ab0..19c0f9d1d88ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q @@ -10,48 +10,48 @@ desc alter_coltype; select count(*) from alter_coltype where dt = '100x'; -- alter partition key column data type for dt column. -alter table alter_coltype partition column (dt int); +-- alter table alter_coltype partition column (dt int); -- load a new partition using new data type. -insert overwrite table alter_coltype partition(dt=10, ts='3.0') select * from src1; +-- insert overwrite table alter_coltype partition(dt=10, ts='3.0') select * from src1; -- make sure the partition predicate still works. -select count(*) from alter_coltype where dt = '100x'; -explain extended select count(*) from alter_coltype where dt = '100x'; +-- select count(*) from alter_coltype where dt = '100x'; +-- explain extended select count(*) from alter_coltype where dt = '100x'; -select count(*) from alter_coltype where dt = 100; +-- select count(*) from alter_coltype where dt = '100'; -- alter partition key column data type for ts column. -alter table alter_coltype partition column (ts double); +-- alter table alter_coltype partition column (ts double); -alter table alter_coltype partition column (dt string); +-- alter table alter_coltype partition column (dt string); -- load a new partition using new data type. -insert overwrite table alter_coltype partition(dt='100x', ts=3.0) select * from src1; +-- insert overwrite table alter_coltype partition(dt='100x', ts=3.0) select * from src1; -- validate partition key column predicate can still work. -select count(*) from alter_coltype where ts = '6:30pm'; -explain extended select count(*) from alter_coltype where ts = '6:30pm'; +-- select count(*) from alter_coltype where ts = '6:30pm'; +-- explain extended select count(*) from alter_coltype where ts = '6:30pm'; -- validate partition key column predicate on two different partition column data type -- can still work. -select count(*) from alter_coltype where ts = 3.0 and dt=10; -explain extended select count(*) from alter_coltype where ts = 3.0 and dt=10; +-- select count(*) from alter_coltype where ts = 3.0 and dt=10; +-- explain extended select count(*) from alter_coltype where ts = 3.0 and dt=10; -- query where multiple partition values (of different datatypes) are being selected -select key, value, dt, ts from alter_coltype where dt is not null; -explain extended select key, value, dt, ts from alter_coltype where dt is not null; +-- select key, value, dt, ts from alter_coltype where dt is not null; +-- explain extended select key, value, dt, ts from alter_coltype where dt is not null; -select count(*) from alter_coltype where ts = 3.0; +-- select count(*) from alter_coltype where ts = 3.0; -- make sure the partition predicate still works. -select count(*) from alter_coltype where dt = '100x' or dt = '10'; -explain extended select count(*) from alter_coltype where dt = '100x' or dt = '10'; +-- select count(*) from alter_coltype where dt = '100x' or dt = '10'; +-- explain extended select count(*) from alter_coltype where dt = '100x' or dt = '10'; -desc alter_coltype; -desc alter_coltype partition (dt='100x', ts='6:30pm'); -desc alter_coltype partition (dt='100x', ts=3.0); -desc alter_coltype partition (dt=10, ts=3.0); +-- desc alter_coltype; +-- desc alter_coltype partition (dt='100x', ts='6:30pm'); +-- desc alter_coltype partition (dt='100x', ts=3.0); +-- desc alter_coltype partition (dt=10, ts=3.0); drop table alter_coltype; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q index 7bcb9f071c415..7a1f3dd51d617 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q @@ -2,10 +2,10 @@ create table if not exists alter_part_protect_mode(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_protect_mode partition (year='1996', month='10'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='12'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1995', month='09'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1994', month='07'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='10'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='12'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1995', month='09'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1994', month='07'); -- offline alter table alter_part_protect_mode partition (year='1996') disable offline; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q index d498cd52a5ffb..8ebbe98824dec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q @@ -4,7 +4,7 @@ DROP TABLE alter_rename_partition; SHOW TABLES; 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 ; +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; @@ -28,7 +28,7 @@ USE alter_rename_partition_db; SHOW TABLES; 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 ; +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; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q index 5a481e7f8ebfe..b870108bddd2b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q @@ -7,7 +7,7 @@ create table alter_varchar2 ( ) partitioned by (hr int); insert overwrite table alter_varchar2 partition (hr=1) - select value from src limit 1; + select value from src tablesample (1 rows); select c1, length(c1) from alter_varchar2; @@ -16,7 +16,9 @@ alter table alter_varchar2 change column c1 c1 varchar(10); select hr, c1, length(c1) from alter_varchar2 where hr = 1; insert overwrite table alter_varchar2 partition (hr=2) - select key from src limit 1; + select key from src tablesample (1 rows); + +set hive.fetch.task.conversion=more; select hr, c1, length(c1) from alter_varchar2 where hr = 1; select hr, c1, length(c1) from alter_varchar2 where hr = 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q index e7053c1c8eb55..5ccd2c8c62dcf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q @@ -1,3 +1,4 @@ +set hive.support.quoted.identifiers=none; -- TOK_ALLCOLREF explain select * from (select a.key, a.* from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; -- DOT diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q new file mode 100644 index 0000000000000..ec973e15969d4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q @@ -0,0 +1,76 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc; + +-- column stats are not COMPLETE, so stats are not updated +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc where state='OH'; + +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- state column has 5 distincts. numRows/countDistincts +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where state='OH'; + +-- not equals comparison shouldn't affect number of rows +-- numRows: 8 rawDataSize: 804 +explain extended select * from loc_orc where state!='OH'; +explain extended select * from loc_orc where state<>'OH'; + +-- nulls are treated as constant equality comparison +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where zip is null; +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where !(zip is not null); + +-- not nulls are treated as inverse of nulls +-- numRows: 7 rawDataSize: 702 +explain extended select * from loc_orc where zip is not null; +-- numRows: 7 rawDataSize: 702 +explain extended select * from loc_orc where !(zip is null); + +-- NOT evaluation. true will pass all rows, false will not pass any rows +-- numRows: 8 rawDataSize: 804 +explain extended select * from loc_orc where !false; +-- numRows: 0 rawDataSize: 0 +explain extended select * from loc_orc where !true; + +-- OR evaluation. 1 row for OH and 1 row for CA +-- numRows: 2 rawDataSize: 204 +explain extended select * from loc_orc where state='OH' or state='CA'; + +-- AND evaluation. cascadingly apply rules. 8/2 = 4/2 = 2 +-- numRows: 2 rawDataSize: 204 +explain extended select * from loc_orc where year=2001 and year is null; +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where year=2001 and state='OH' and state='FL'; + +-- AND and OR together. left expr will yield 1 row and right will yield 1 row +-- numRows: 3 rawDataSize: 306 +explain extended select * from loc_orc where (year=2001 and year is null) or (state='CA'); + +-- AND and OR together. left expr will yield 8 rows and right will yield 1 row +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where (year=2001 or year is null) and (state='CA'); + +-- all inequality conditions rows/3 is the rules +-- numRows: 2 rawDataSize: 204 +explain extended select * from loc_orc where locid < 30; +explain extended select * from loc_orc where locid > 30; +explain extended select * from loc_orc where locid <= 30; +explain extended select * from loc_orc where locid >= 30; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q new file mode 100644 index 0000000000000..05cb036b466a2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q @@ -0,0 +1,69 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc; + +-- partial column stats +analyze table loc_orc compute statistics for columns state; + +-- inner group by: map - numRows: 8 reduce - numRows: 4 +-- outer group by: map - numRows: 4 reduce numRows: 2 +explain extended select a, c, min(b) +from ( select state as a, locid as b, count(*) as c + from loc_orc + group by state,locid + ) sq1 +group by a,c; + +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- only one distinct value in year column + 1 NULL value +-- map-side GBY: numRows: 8 (map-side will not do any reduction) +-- reduce-side GBY: numRows: 2 +explain extended select year from loc_orc group by year; + +-- map-side GBY: numRows: 8 +-- reduce-side GBY: numRows: 4 +explain extended select state,locid from loc_orc group by state,locid; + +-- map-side GBY numRows: 32 reduce-side GBY numRows: 16 +explain extended select state,locid from loc_orc group by state,locid with cube; + +-- map-side GBY numRows: 24 reduce-side GBY numRows: 12 +explain extended select state,locid from loc_orc group by state,locid with rollup; + +-- map-side GBY numRows: 8 reduce-side GBY numRows: 4 +explain extended select state,locid from loc_orc group by state,locid grouping sets((state)); + +-- map-side GBY numRows: 16 reduce-side GBY numRows: 8 +explain extended select state,locid from loc_orc group by state,locid grouping sets((state),(locid)); + +-- map-side GBY numRows: 24 reduce-side GBY numRows: 12 +explain extended select state,locid from loc_orc group by state,locid grouping sets((state),(locid),()); + +-- map-side GBY numRows: 32 reduce-side GBY numRows: 16 +explain extended select state,locid from loc_orc group by state,locid grouping sets((state,locid),(state),(locid),()); + +set hive.stats.map.parallelism=10; + +-- map-side GBY: numRows: 80 (map-side will not do any reduction) +-- reduce-side GBY: numRows: 2 Reason: numDistinct of year is 2. numRows = min(80/2, 2) +explain extended select year from loc_orc group by year; + +-- map-side GBY numRows: 320 reduce-side GBY numRows: 42 Reason: numDistinct of state and locid are 6,7 resp. numRows = min(320/2, 6*7) +explain extended select state,locid from loc_orc group by state,locid with cube; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q new file mode 100644 index 0000000000000..965b0b7ed0a3e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q @@ -0,0 +1,81 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists emp_staging ( + lastname string, + deptid int +) row format delimited fields terminated by '|' stored as textfile; + +create table if not exists dept_staging ( + deptid int, + deptname string +) row format delimited fields terminated by '|' stored as textfile; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table if not exists emp_orc like emp_staging; +alter table emp_orc set fileformat orc; + +create table if not exists dept_orc like dept_staging; +alter table dept_orc set fileformat orc; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +LOAD DATA LOCAL INPATH '../../data/files/emp.txt' OVERWRITE INTO TABLE emp_staging; +LOAD DATA LOCAL INPATH '../../data/files/dept.txt' OVERWRITE INTO TABLE dept_staging; +LOAD DATA LOCAL INPATH '../../data/files/loc.txt' OVERWRITE INTO TABLE loc_staging; + +insert overwrite table emp_orc select * from emp_staging; +insert overwrite table dept_orc select * from dept_staging; +insert overwrite table loc_orc select * from loc_staging; + +analyze table emp_orc compute statistics for columns lastname,deptid; +analyze table dept_orc compute statistics for columns deptname,deptid; +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- number of rows +-- emp_orc - 6 +-- dept_orc - 4 +-- loc_orc - 8 + +-- count distincts for relevant columns (since count distinct values are approximate in some cases count distint values will be greater than number of rows) +-- emp_orc.deptid - 3 +-- emp_orc.lastname - 7 +-- dept_orc.deptid - 6 +-- dept_orc.deptname - 5 +-- loc_orc.locid - 6 +-- loc_orc.state - 7 + +-- Expected output rows: 4 +-- Reason: #rows = (6*4)/max(3,6) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid); + +-- 3 way join +-- Expected output rows: 4 +-- Reason: #rows = (6*4*6)/max(3,6)*max(6,3) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid) join emp_orc e1 on (e.deptid = e1.deptid); + +-- Expected output rows: 5 +-- Reason: #rows = (6*4*8)/max(3,6)*max(6,6) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid) join loc_orc l on (e.deptid = l.locid); + +-- join keys of different types +-- Expected output rows: 4 +-- Reason: #rows = (6*4*8)/max(3,6)*max(6,7) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid) join loc_orc l on (e.deptid = l.state); + +-- multi-attribute join +-- Expected output rows: 0 +-- Reason: #rows = (6*4)/max(3,6)*max(7,5) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid and e.lastname = d.deptname); + +-- 3 way and multi-attribute join +-- Expected output rows: 0 +-- Reason: #rows = (6*4*8)/max(3,6)*max(7,5)*max(3,6)*max(7,7) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid and e.lastname = d.deptname) join loc_orc l on (e.deptid = l.locid and e.lastname = l.state); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q new file mode 100644 index 0000000000000..0a9f880b5f31e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q @@ -0,0 +1,30 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +analyze table loc_orc compute statistics for columns state, locid, zip, year; + +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc; + +-- numRows: 4 rawDataSize: 396 +explain extended select * from loc_orc limit 4; + +-- greater than the available number of rows +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc limit 16; + +-- numRows: 0 rawDataSize: 0 +explain extended select * from loc_orc limit 0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q new file mode 100644 index 0000000000000..839c7d84962fc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q @@ -0,0 +1,85 @@ +set hive.stats.fetch.column.stats=true; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year string +) row format delimited fields terminated by '|' stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/loc.txt' OVERWRITE INTO TABLE loc_staging; + +create table if not exists loc_orc ( + state string, + locid int, + zip bigint +) partitioned by(year string) stored as orc; + +-- basicStatState: NONE colStatState: NONE +explain extended select * from loc_orc; + +insert overwrite table loc_orc partition(year) select * from loc_staging; + +-- stats are disabled. basic stats will report the file size but not raw data size. so initial statistics will be PARTIAL + +-- basicStatState: PARTIAL colStatState: NONE +explain extended select * from loc_orc; + +-- partition level analyze statistics for specific parition +analyze table loc_orc partition(year='2001') compute statistics; + +-- basicStatState: PARTIAL colStatState: NONE +explain extended select * from loc_orc where year='__HIVE_DEFAULT_PARTITION__'; + +-- basicStatState: PARTIAL colStatState: NONE +explain extended select * from loc_orc; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from loc_orc where year='2001'; + +-- partition level analyze statistics for all partitions +analyze table loc_orc partition(year) compute statistics; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from loc_orc where year='__HIVE_DEFAULT_PARTITION__'; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from loc_orc; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from loc_orc where year='2001' or year='__HIVE_DEFAULT_PARTITION__'; + +-- both partitions will be pruned +-- basicStatState: NONE colStatState: NONE +explain extended select * from loc_orc where year='2001' and year='__HIVE_DEFAULT_PARTITION__'; + +-- partition level partial column statistics +analyze table loc_orc partition(year='2001') compute statistics for columns state,locid; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select zip from loc_orc; + +-- basicStatState: COMPLETE colStatState: PARTIAL +explain extended select state from loc_orc; + +-- column statistics for __HIVE_DEFAULT_PARTITION__ is not supported yet. Hence colStatState reports PARTIAL +-- basicStatState: COMPLETE colStatState: PARTIAL +explain extended select state,locid from loc_orc; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select state,locid from loc_orc where year='2001'; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select state,locid from loc_orc where year!='2001'; + +-- basicStatState: COMPLETE colStatState: PARTIAL +explain extended select * from loc_orc; + +-- This is to test filter expression evaluation on partition column +-- numRows: 2 dataSize: 8 basicStatState: COMPLETE colStatState: COMPLETE +explain extended select locid from loc_orc where locid>0 and year='2001'; +explain extended select locid,year from loc_orc where locid>0 and year='2001'; +explain extended select * from (select locid,year from loc_orc) test where locid>0 and year='2001'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q new file mode 100644 index 0000000000000..5fc3f64b90180 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q @@ -0,0 +1,143 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists alltypes ( + bo1 boolean, + ti1 tinyint, + si1 smallint, + i1 int, + bi1 bigint, + f1 float, + d1 double, + de1 decimal, + ts1 timestamp, + da1 timestamp, + s1 string, + vc1 varchar(5), + m1 map, + l1 array, + st1 struct +) row format delimited fields terminated by '|' +collection items terminated by ',' +map keys terminated by ':' stored as textfile; + +create table alltypes_orc like alltypes; +alter table alltypes_orc set fileformat orc; + +load data local inpath '../../data/files/alltypes.txt' overwrite into table alltypes; + +insert overwrite table alltypes_orc select * from alltypes; + +-- basicStatState: COMPLETE colStatState: NONE numRows: 2 rawDataSize: 1514 +explain extended select * from alltypes_orc; + +-- statistics for complex types are not supported yet +analyze table alltypes_orc compute statistics for columns bo1, ti1, si1, i1, bi1, f1, d1, s1, vc1; + +-- numRows: 2 rawDataSize: 1514 +explain extended select * from alltypes_orc; + +-- numRows: 2 rawDataSize: 8 +explain extended select bo1 from alltypes_orc; + +-- col alias renaming +-- numRows: 2 rawDataSize: 8 +explain extended select i1 as int1 from alltypes_orc; + +-- numRows: 2 rawDataSize: 174 +explain extended select s1 from alltypes_orc; + +-- column statistics for complex types unsupported and so statistics will not be updated +-- numRows: 2 rawDataSize: 1514 +explain extended select m1 from alltypes_orc; + +-- numRows: 2 rawDataSize: 246 +explain extended select bo1, ti1, si1, i1, bi1, f1, d1,s1 from alltypes_orc; + +-- numRows: 2 rawDataSize: 0 +explain extended select null from alltypes_orc; + +-- numRows: 2 rawDataSize: 8 +explain extended select 11 from alltypes_orc; + +-- numRows: 2 rawDataSize: 16 +explain extended select 11L from alltypes_orc; + +-- numRows: 2 rawDataSize: 16 +explain extended select 11.0 from alltypes_orc; + +-- numRows: 2 rawDataSize: 178 +explain extended select "hello" from alltypes_orc; +explain extended select cast("hello" as char(5)) from alltypes_orc; +explain extended select cast("hello" as varchar(5)) from alltypes_orc; + +-- numRows: 2 rawDataSize: 96 +explain extended select unbase64("0xe23") from alltypes_orc; + +-- numRows: 2 rawDataSize: 16 +explain extended select cast("1" as TINYINT), cast("20" as SMALLINT) from alltypes_orc; + +-- numRows: 2 rawDataSize: 80 +explain extended select cast("1970-12-31 15:59:58.174" as TIMESTAMP) from alltypes_orc; + +-- numRows: 2 rawDataSize: 112 +explain extended select cast("1970-12-31 15:59:58.174" as DATE) from alltypes_orc; + +-- numRows: 2 rawDataSize: 224 +explain extended select cast("58.174" as DECIMAL) from alltypes_orc; + +-- numRows: 2 rawDataSize: 112 +explain extended select array(1,2,3) from alltypes_orc; + +-- numRows: 2 rawDataSize: 1508 +explain extended select str_to_map("a=1 b=2 c=3", " ", "=") from alltypes_orc; + +-- numRows: 2 rawDataSize: 112 +explain extended select NAMED_STRUCT("a", 11, "b", 11) from alltypes_orc; + +-- numRows: 2 rawDataSize: 250 +explain extended select CREATE_UNION(0, "hello") from alltypes_orc; + +-- COUNT(*) is projected as new column. It is not projected as GenericUDF and so datasize estimate will be based on number of rows +-- numRows: 1 rawDataSize: 8 +explain extended select count(*) from alltypes_orc; + +-- COUNT(1) is projected as new column. It is not projected as GenericUDF and so datasize estimate will be based on number of rows +-- numRows: 1 rawDataSize: 8 +explain extended select count(1) from alltypes_orc; + +-- column statistics for complex column types will be missing. data size will be calculated from available column statistics +-- numRows: 2 rawDataSize: 254 +explain extended select *,11 from alltypes_orc; + +-- subquery selects +-- inner select - numRows: 2 rawDataSize: 8 +-- outer select - numRows: 2 rawDataSize: 8 +explain extended select i1 from (select i1 from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 16 +-- outer select - numRows: 2 rawDataSize: 8 +explain extended select i1 from (select i1,11 from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 16 +-- outer select - numRows: 2 rawDataSize: 186 +explain extended select i1,"hello" from (select i1,11 from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 24 +-- outer select - numRows: 2 rawDataSize: 16 +explain extended select x from (select i1,11.0 as x from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 104 +-- outer select - numRows: 2 rawDataSize: 186 +explain extended select x,"hello" from (select i1 as x, unbase64("0xe23") as ub from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 186 +-- middle select - numRows: 2 rawDataSize: 178 +-- outer select - numRows: 2 rawDataSize: 194 +explain extended select h, 11.0 from (select hell as h from (select i1, "hello" as hell from alltypes_orc limit 10) in1 limit 10) in2; + +-- This test is for FILTER operator where filter expression is a boolean column +-- numRows: 2 rawDataSize: 8 +explain extended select bo1 from alltypes_orc where bo1; + +-- numRows: 0 rawDataSize: 0 +explain extended select bo1 from alltypes_orc where !bo1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q new file mode 100644 index 0000000000000..4140fe610d7cd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q @@ -0,0 +1,53 @@ +set hive.stats.fetch.column.stats=true; +set hive.stats.autogather=false; + +create table if not exists emp_staging ( + lastname string, + deptid int +) row format delimited fields terminated by '|' stored as textfile; + +create table if not exists emp_orc like emp_staging; +alter table emp_orc set fileformat orc; + +-- basicStatState: NONE colStatState: NONE +explain extended select * from emp_orc; + +LOAD DATA LOCAL INPATH '../../data/files/emp.txt' OVERWRITE INTO TABLE emp_staging; + +insert overwrite table emp_orc select * from emp_staging; + +-- stats are disabled. basic stats will report the file size but not raw data size. so initial statistics will be PARTIAL + +-- basicStatState: PARTIAL colStatState: NONE +explain extended select * from emp_orc; + +-- table level analyze statistics +analyze table emp_orc compute statistics; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from emp_orc; + +-- column level partial statistics +analyze table emp_orc compute statistics for columns deptid; + +-- basicStatState: COMPLETE colStatState: PARTIAL +explain extended select * from emp_orc; + +-- all selected columns have statistics +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select deptid from emp_orc; + +-- column level complete statistics +analyze table emp_orc compute statistics for columns lastname,deptid; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select * from emp_orc; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select lastname from emp_orc; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select deptid from emp_orc; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select lastname,deptid from emp_orc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q new file mode 100644 index 0000000000000..586d9e1e2c023 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q @@ -0,0 +1,55 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- numRows: 8 rawDataSize: 688 +explain extended select state from loc_orc; + +-- numRows: 16 rawDataSize: 1376 +explain extended select * from (select state from loc_orc union all select state from loc_orc) tmp; + +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc; + +-- numRows: 16 rawDataSize: 1592 +explain extended select * from (select * from loc_orc union all select * from loc_orc) tmp; + +create database test; +use test; +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +analyze table loc_staging compute statistics; +analyze table loc_staging compute statistics for columns state,locid,zip,year; +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- numRows: 16 rawDataSize: 1376 +explain extended select * from (select state from default.loc_orc union all select state from test.loc_orc) temp; + +-- numRows: 16 rawDataSize: 1376 +explain extended select * from (select state from test.loc_staging union all select state from test.loc_orc) temp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q new file mode 100644 index 0000000000000..3788301ebb3d2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q @@ -0,0 +1,13 @@ + +set hive.compat=latest; + +-- With ansi sql arithmetic enabled, int / int => exact numeric type +explain select cast(key as int) / cast(key as int) from src limit 1; +select cast(key as int) / cast(key as int) from src limit 1; + + +set hive.compat=0.12; + +-- With ansi sql arithmetic disabled, int / int => double +explain select cast(key as int) / cast(key as int) from src limit 1; +select cast(key as int) / cast(key as int) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q index b83eab5d38137..cc9801d8871b8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q @@ -14,7 +14,7 @@ create table tstsrcpart like srcpart; -- to be thrown during the LOAD step. This behavior is now tested in -- clientnegative/archive_corrupt.q -load data local inpath '../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11'); +load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', 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'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q index ddc06a99bb5bc..50c0faa5e4108 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q @@ -1,7 +1,7 @@ set hive.archive.enabled = true; set hive.enforce.bucketing = true; --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20) drop table tstsrc; drop table tstsrcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q new file mode 100644 index 0000000000000..79ae17ad5da2c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q @@ -0,0 +1,36 @@ +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; + +create table src_autho_test (key STRING, value STRING) ; + +set hive.security.authorization.enabled=true; +set role ADMIN; +--table grant to user + +grant select on table src_autho_test to user user_sauth; + +show grant user user_sauth on table src_autho_test; + + +revoke select on table src_autho_test from user user_sauth; +show grant user user_sauth on table src_autho_test; + +--role +create role src_role; +grant role src_role to user user_sauth; +show role grant user user_sauth; + +--table grant to role + +grant select on table src_autho_test to role src_role; + +show grant role src_role on table src_autho_test; +revoke select on table src_autho_test from role src_role; + +-- drop role +drop role src_role; + +set hive.security.authorization.enabled=false; +drop table src_autho_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q index 4fc79b9ed294a..3353c534e1793 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q @@ -6,7 +6,7 @@ ALTER TABLE authorization_part SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="T set hive.security.authorization.enabled=true; -- column grant to user -grant Create on table authorization_part to user hive_test_user; +grant Create on authorization_part to user hive_test_user; grant Update on table authorization_part to user hive_test_user; grant Drop on table authorization_part to user hive_test_user; grant select on table src_auth_tmp to user hive_test_user; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q new file mode 100644 index 0000000000000..1abe659fa4476 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q @@ -0,0 +1,17 @@ +-- SORT_BEFORE_DIFF + +create table dummy (key string, value string); + +grant select on database default to user hive_test_user; +grant select on table dummy to user hive_test_user; +grant select (key, value) on table dummy to user hive_test_user; + +show grant user hive_test_user on database default; +show grant user hive_test_user on table dummy; +show grant user hive_test_user on all; + +grant select on database default to user hive_test_user2; +grant select on table dummy to user hive_test_user2; +grant select (key, value) on table dummy to user hive_test_user2; + +show grant on all; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q new file mode 100644 index 0000000000000..45c4a7dc85b72 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q @@ -0,0 +1,17 @@ +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_test_user; + +-- actions from admin should work as if admin has all privileges + +create table t1(i int); +set user.name=hive_admin_user; + +show current roles; +set role ADMIN; +show current roles; +select * from t1; +grant all on table t1 to user user1; +show grant user user1 on table t1; +drop table t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q new file mode 100644 index 0000000000000..65a7b339c2b76 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q @@ -0,0 +1,14 @@ +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; + +-- admin required for create function +set role ADMIN; + +create temporary function temp_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; +create function perm_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; + +drop temporary function temp_fn; +drop function perm_fn; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q new file mode 100644 index 0000000000000..fb60500b899ec --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q @@ -0,0 +1,12 @@ +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; + +-- admin required for create macro +set role ADMIN; + +create temporary macro mymacro1(x double) x * x; + +drop temporary macro mymacro1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q new file mode 100644 index 0000000000000..17f4861cd20f8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q @@ -0,0 +1,10 @@ +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; + +create table create_table_creator_priv_test(i int); + +-- all privileges should have been set for user + +show grant user user1 on table create_table_creator_priv_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q new file mode 100644 index 0000000000000..8473178cd6607 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q @@ -0,0 +1,18 @@ +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 t_gpr1(i int); + +-- all privileges should have been set for user + +GRANT ALL ON t_gpr1 TO ROLE public; + +SHOW GRANT ON TABLE t_gpr1; + +set user.name=user2; +SHOW CURRENT ROLES; +-- user2 should be able to do a describe table, as pubic is in the current roles +DESC t_gpr1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q new file mode 100644 index 0000000000000..02d364edb488d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q @@ -0,0 +1,43 @@ +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_priv1(i int); + +-- all privileges should have been set for user + +-- grant insert privilege to another user +GRANT INSERT ON table_priv1 TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv1; + +-- grant select privilege to another user with grant +GRANT SELECT ON table_priv1 TO USER user2 with grant option; +SHOW GRANT USER user2 ON TABLE table_priv1; + +set user.name=user2; +-- change to other user - user2 +-- grant permissions to another user as user2 +GRANT SELECT ON table_priv1 TO USER user3 with grant option; +SHOW GRANT USER user3 ON TABLE table_priv1; + +set user.name=user3; +-- change to other user - user3 +-- grant permissions to another user as user3 +GRANT SELECT ON table_priv1 TO USER user4 with grant option; +SHOW GRANT USER user4 ON TABLE table_priv1; + +set user.name=user1; +-- switched back to table owner + +-- grant all with grant to user22 +GRANT ALL ON table_priv1 TO USER user22 with grant option; +SHOW GRANT USER user22 ON TABLE table_priv1; + +set user.name=user22; + +-- grant all without grant to user33 +GRANT ALL ON table_priv1 TO USER user33 with grant option; +SHOW GRANT USER user33 ON TABLE table_priv1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q new file mode 100644 index 0000000000000..1f177ffd1fadb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q @@ -0,0 +1,12 @@ +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.stats.dbclass=fs; +set hive.security.authorization.enabled=true; +create table t1 (a int); +create index t1_index on table t1(a) as 'COMPACT' WITH DEFERRED REBUILD; +desc formatted default__t1_t1_index__; +alter index t1_index on t1 rebuild; + +drop table t1; + +set hive.security.authorization.enabled=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q new file mode 100644 index 0000000000000..85d8b1114b010 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q @@ -0,0 +1,16 @@ +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; + +-- actions that require user to be table owner +create table t1(i int); + +ALTER TABLE t1 SET SERDEPROPERTIES ('field.delim' = ','); +drop table t1; + +create table t1(i int); +create view vt1 as select * from t1; + +drop view vt1; +alter table t1 rename to tnew1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q new file mode 100644 index 0000000000000..36ab2600dcd2d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q @@ -0,0 +1,21 @@ +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; + +set role admin; +-- create role, db, make role the owner of db +create role testrole; +grant role testrole to user hrt_1; +create database testdb; +alter database testdb set owner role testrole; +desc database testdb; + +-- actions that require user to be db owner +-- create table +use testdb; +create table foobar (foo string, bar string); + +-- drop db +drop database testdb cascade; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q new file mode 100644 index 0000000000000..bee091b1d349c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q @@ -0,0 +1,19 @@ +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_part1; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part2; + + + + +-- check add partition without insert privilege +create table tpart(i int, j int) partitioned by (k string); + +alter table tpart add partition (k = '1') location '${system:test.tmp.dir}/a_uri_add_part1/'; +alter table tpart add partition (k = '2') location '${system:test.tmp.dir}/a_uri_add_part2/'; + +select count(*) from tpart; + +analyze table tpart partition (k) compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q new file mode 100644 index 0000000000000..ccda3b5157979 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q @@ -0,0 +1,61 @@ +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_rev(i int); + +-- grant insert privilege to user2 +GRANT INSERT ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; +SHOW GRANT USER user2 ON ALL; + +-- revoke insert privilege from user2 +REVOKE INSERT ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- grant all privileges one at a time -- +-- grant insert privilege to user2 +GRANT INSERT ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; +SHOW GRANT USER user2 ON ALL; + +-- grant select privilege to user2, with grant option +GRANT SELECT ON table_priv_rev TO USER user2 WITH GRANT OPTION; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- grant update privilege to user2 +GRANT UPDATE ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- grant delete privilege to user2 +GRANT DELETE ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + + +-- start revoking -- +-- revoke update privilege from user2 +REVOKE UPDATE ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; +SHOW GRANT USER user2 ON ALL; + +-- revoke DELETE privilege from user2 +REVOKE DELETE ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- revoke insert privilege from user2 +REVOKE INSERT ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- revoke select privilege from user2 +REVOKE SELECT ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; +SHOW GRANT USER user2 ON ALL; + +-- grant all followed by revoke all +GRANT ALL ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +REVOKE ALL ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q new file mode 100644 index 0000000000000..f89d0dc985688 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q @@ -0,0 +1,38 @@ +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; + +-- enable sql standard authorization +-- role granting without role keyword +set role ADMIN; +create role src_role2; +grant src_role2 to user user2 ; +show role grant user user2; +show roles; + +-- revoke role without role keyword +revoke src_role2 from user user2; +show role grant user user2; +show roles; + +---------------------------------------- +-- role granting without role keyword, with admin option (syntax check) +---------------------------------------- + +create role src_role_wadmin; +grant src_role_wadmin to user user2 with admin option; +show role grant user user2; + +-- revoke role without role keyword +revoke src_role_wadmin from user user2; +show role grant user user2; + + + +-- drop roles +show roles; +drop role src_role2; +show roles; +drop role src_role_wadmin; +show roles; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q new file mode 100644 index 0000000000000..984d7ed1d091a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q @@ -0,0 +1,34 @@ +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.cli.print.header=true; +set user.name=hive_admin_user; +set role ADMIN; + +---------------------------------------- +-- role granting with admin option +---------------------------------------- + +create role src_role_wadmin; +grant src_role_wadmin to user user2 with admin option; +show role grant user user2; +show principals src_role_wadmin; + +set user.name=user2; +set role src_role_wadmin; +grant src_role_wadmin to user user3; +show role grant user user3; + +set user.name=hive_admin_user; +set role ADMIN; +show principals src_role_wadmin; + +set user.name=user2; +set role src_role_wadmin; +revoke src_role_wadmin from user user3; +show role grant user user3; + +set user.name=hive_admin_user; +set role ADMIN; +show principals src_role_wadmin; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q new file mode 100644 index 0000000000000..6b5af6e94e16a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q @@ -0,0 +1,21 @@ +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; +grant role r1 to user hive_admin_user; +set role r1; +show current roles; + +set role PUBLIC; +show current roles; + +set role ALL; +show current roles; + +set role ADMIN; +drop role r1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q new file mode 100644 index 0000000000000..bd7bbfedf83e8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q @@ -0,0 +1,66 @@ +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; + +-- Test view authorization , and 'show grant' variants + +create table t1(i int, j int, k int); +show grant on table t1; + +-- protecting certain columns +create view vt1 as select i,k from t1; + +-- protecting certain rows +create view vt2 as select * from t1 where i > 1; + +show grant user user1 on all; + +--view grant to user +-- try with and without table keyword + +grant select on vt1 to user user2; +grant insert on table vt1 to user user3; + +show grant user user2 on table vt1; +show grant user user3 on table vt1; + + +set user.name=user2; +select * from vt1; + +set user.name=user1; + +grant all on table vt2 to user user2; +show grant user user2 on table vt2; +show grant user user2 on all; + +revoke all on vt2 from user user2; +show grant user user2 on table vt2; + +show grant on table vt2; + + +revoke select on table vt1 from user user2; +show grant user user2 on table vt1; + +show grant user user2 on all; + +-- grant privileges on roles for view, after next statement +show grant user user3 on table vt1; + +set user.name=hive_admin_user; +show current roles; +set role ADMIN; +create role role_v; +grant role_v to user user4 ; +show role grant user user4; +show roles; + +grant all on table vt2 to role role_v; +show grant role role_v on table vt2; + +revoke delete on table vt2 from role role_v; +show grant role role_v on table vt2; +show grant on table vt2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q index eaf7489a17025..b8734abfd12ac 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q @@ -1,3 +1,5 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; + set hive.auto.convert.join = true; set hive.mapjoin.localtask.max.memory.usage = 0.0001; set hive.mapjoin.check.memory.rows = 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q index 289bfbc6da917..e7846eeecf5c3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q @@ -32,10 +32,10 @@ from studenttab10k_smb s join votertab10k_smb v on (s.name = v.name) group by s.name; -load data local inpath '../data/files/empty1.txt' into table studenttab10k_smb; -load data local inpath '../data/files/empty2.txt' into table studenttab10k_smb; -load data local inpath '../data/files/empty1.txt' into table votertab10k_smb; -load data local inpath '../data/files/empty2.txt' into table votertab10k_smb; +load data local inpath '../../data/files/empty1.txt' into table studenttab10k_smb; +load data local inpath '../../data/files/empty2.txt' into table studenttab10k_smb; +load data local inpath '../../data/files/empty1.txt' into table votertab10k_smb; +load data local inpath '../../data/files/empty2.txt' into table votertab10k_smb; explain select s.name, count(distinct registration) from studenttab10k_smb s join votertab10k_smb v @@ -51,10 +51,10 @@ group by s.name; create table studenttab10k_part (name string, age int, gpa double) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets; create table votertab10k_part (name string, age int, registration string, contributions float) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets; -load data local inpath '../data/files/empty1.txt' into table studenttab10k_part partition (p='foo'); -load data local inpath '../data/files/empty2.txt' into table studenttab10k_part partition (p='foo'); -load data local inpath '../data/files/empty1.txt' into table votertab10k_part partition (p='foo'); -load data local inpath '../data/files/empty2.txt' into table votertab10k_part partition (p='foo'); +load data local inpath '../../data/files/empty1.txt' into table studenttab10k_part partition (p='foo'); +load data local inpath '../../data/files/empty2.txt' into table studenttab10k_part partition (p='foo'); +load data local inpath '../../data/files/empty1.txt' into table votertab10k_part partition (p='foo'); +load data local inpath '../../data/files/empty2.txt' into table votertab10k_part partition (p='foo'); explain select s.name, count(distinct registration) from studenttab10k_part s join votertab10k_part v diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q index 458504cdc3d04..eefd2111c97b4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q @@ -1,7 +1,7 @@ set hive.auto.convert.join = true; CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in3.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in3.txt' INTO TABLE myinput1; SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value; SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value; @@ -38,10 +38,10 @@ SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN my CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2; SET hive.optimize.bucketmapjoin = true; SET hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q index 766348d7c001e..d1b7bb40189a4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q @@ -1,7 +1,7 @@ set hive.auto.convert.join = true; CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in1.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' INTO TABLE myinput1; SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b; SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q index 46a4a0d5339a7..55bd975803d53 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q @@ -1,13 +1,13 @@ -- HIVE-5056 RS has expression list for values, but it's ignored in MapJoinProcessor create table testsrc ( `key` int,`val` string); -load data local inpath '../data/files/kv1.txt' overwrite into table testsrc; +load data local inpath '../../data/files/kv1.txt' overwrite into table testsrc; drop table if exists orderpayment_small; create table orderpayment_small (`dealid` int,`date` string,`time` string, `cityid` int, `userid` int); -insert overwrite table orderpayment_small select 748, '2011-03-24', '2011-03-24', 55 ,5372613 from testsrc limit 1; +insert overwrite table orderpayment_small select 748, '2011-03-24', '2011-03-24', 55 ,5372613 from testsrc tablesample (1 rows); drop table if exists user_small; create table user_small( userid int); -insert overwrite table user_small select key from testsrc limit 100; +insert overwrite table user_small select key from testsrc tablesample (100 rows); set hive.auto.convert.join.noconditionaltask.size = 200; explain extended SELECT diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q new file mode 100644 index 0000000000000..f23e227f1ec90 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q @@ -0,0 +1,29 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; +set hive.auto.convert.join=true; +set hive.auto.convert.join.use.nonstaged=true; + +set hive.auto.convert.join.noconditionaltask.size=100; + +explain +select a.* from src a join src b on a.key=b.key limit 40; + +select a.* from src a join src b on a.key=b.key limit 40; + +explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40; + +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40; + +set hive.auto.convert.join.noconditionaltask.size=100; + +explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + +set hive.mapjoin.localtask.max.memory.usage = 0.0001; +set hive.mapjoin.check.memory.rows = 2; + +-- fallback to common join +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q index e76b560f41899..ddd2c1882ec4d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q @@ -2,19 +2,19 @@ CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); - -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q index f9fa1e4d4e1db..da2e26fde7069 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q @@ -1,19 +1,19 @@ -- small 1 part, 2 bucket & big 2 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); - -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q index db53a656a850a..f434b33603603 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q @@ -2,19 +2,19 @@ CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; @@ -23,9 +23,9 @@ set hive.optimize.bucketmapjoin.sortedmerge = true; CREATE TABLE bucket_medium (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); explain extended select count(*) FROM bucket_small a JOIN bucket_medium b ON a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN bucket_medium d ON c.key = b.key; select count(*) FROM bucket_small a JOIN bucket_medium b ON a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN bucket_medium d ON c.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q new file mode 100644 index 0000000000000..7e9555d7917df --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q @@ -0,0 +1,92 @@ +set hive.auto.convert.join=true; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; + +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +CREATE TABLE stage_bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (file_tag STRING); + +CREATE TABLE bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (day STRING, pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile; + +CREATE TABLE stage_bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (file_tag STRING); + +CREATE TABLE bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile; + +load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='1'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='2'); + +insert overwrite table bucket_small partition(pri) +select +key, +value, +file_tag as pri +from +stage_bucket_small +where file_tag between 1 and 2; + +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' overwrite into table stage_bucket_big partition (file_tag='1'); + +insert overwrite table bucket_big partition(day,pri) +select +key, +value, +'day1' as day, +1 as pri +from +stage_bucket_big +where +file_tag='1'; + +select +a.key , +a.value , +b.value , +'day1' as day, +1 as pri +from +( +select +key, +value +from bucket_big where day='day1' +) a +left outer join +( +select +key, +value +from bucket_small +where pri between 1 and 2 +) b +on +(a.key = b.key) +; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q index 75339778af802..eef5483b5347d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q @@ -1,16 +1,16 @@ -- small 1 part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q index 39a695fdf039d..c094ecdb6be5e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q @@ -1,16 +1,16 @@ -- small 2 part, 2 bucket & big 1 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q index 6072272c478a3..18acfbfb76543 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q @@ -1,18 +1,18 @@ -- small 2 part, 4 bucket & big 1 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q index a28ce3de5fc63..98d6df9b19591 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q @@ -1,13 +1,13 @@ -- small no part, 4 bucket & big no part, 2 bucket CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small; CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; set hive.auto.convert.sortmerge.join=true; set hive.optimize.bucketmapjoin = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q index d62f6377218c6..e19cc317f36e3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q @@ -1,21 +1,21 @@ -- small 2 part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q index 6302a1be1477c..a66806f21a8a3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q @@ -1,21 +1,21 @@ -- small 2 part, 2 bucket & big 2 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); - -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q index 8367206231974..cb6f173ccfa76 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q @@ -35,7 +35,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ ] }'); -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors4; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors4; set hive.exec.compress.output=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q index 3fe8ff12b4506..f723cbcc608fb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q @@ -36,7 +36,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE doctors_with_new_field; -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors_with_new_field; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors_with_new_field; SELECT * FROM doctors_with_new_field ORDER BY first_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q index 25b77c06270d0..4c33a834668b0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q @@ -37,7 +37,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE doctors4; -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors4; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors4; CREATE TABLE episodes ROW FORMAT @@ -70,7 +70,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE episodes; -LOAD DATA LOCAL INPATH '../data/files/episodes.avro' INTO TABLE episodes; +LOAD DATA LOCAL INPATH '../../data/files/episodes.avro' INTO TABLE episodes; SELECT e.title, e.air_date, d.first_name, d.last_name, d.extra_field, e.air_date FROM doctors4 d JOIN episodes e ON (d.number=e.doctor) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q index 584c6f740ba91..f90ceb96f5db9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q @@ -17,7 +17,7 @@ CREATE TABLE test_serializer(string1 STRING, ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' COLLECTION ITEMS TERMINATED BY ':' MAP KEYS TERMINATED BY '#' LINES TERMINATED BY '\n' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/csv.txt' INTO TABLE test_serializer; +LOAD DATA LOCAL INPATH '../../data/files/csv.txt' INTO TABLE test_serializer; CREATE TABLE as_avro ROW FORMAT diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q index 8e4d40f2bd59b..6fe5117026ce8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q @@ -28,7 +28,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ ] }'); -LOAD DATA LOCAL INPATH '../data/files/episodes.avro' INTO TABLE episodes; +LOAD DATA LOCAL INPATH '../../data/files/episodes.avro' INTO TABLE episodes; CREATE TABLE episodes_partitioned PARTITIONED BY (doctor_pt INT) @@ -66,7 +66,7 @@ INSERT OVERWRITE TABLE episodes_partitioned PARTITION (doctor_pt) SELECT title, SELECT * FROM episodes_partitioned WHERE doctor_pt > 6 ORDER BY air_date; -- Verify that Fetch works in addition to Map -SELECT * FROM episodes_partitioned LIMIT 5; +SELECT * FROM episodes_partitioned ORDER BY air_date LIMIT 5; -- Fetch w/filter to specific partition SELECT * FROM episodes_partitioned WHERE doctor_pt = 6; -- Fetch w/non-existant partition diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q index e3f8b07b30c59..dbb999503b60f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q @@ -30,7 +30,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE doctors; -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors; SELECT * FROM doctors ORDER BY number; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q index e0a8b95401d0b..4f80dc33c9666 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q @@ -1 +1,3 @@ -select cast(cast('a' as binary) as string) from src limit 1; +set hive.fetch.task.conversion=more; + +select cast(cast('a' as binary) as string) from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q index eadf07d14d756..1f5c98a23918f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q @@ -3,6 +3,7 @@ drop table ba_test; -- Everything in ba_table1.q + columnar serde in RCFILE. create table ba_test (ba_key binary, ba_val binary) stored as rcfile; +alter table ba_test set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; describe extended ba_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q index a98a2305cfcaf..39c1d25e73397 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q @@ -3,7 +3,7 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY '9' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE mytable; +LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE mytable; EXPLAIN SELECT REGEXP_REPLACE(REGEXP_REPLACE(REGEXP_REPLACE(key, '\001', '^A'), '\0', '^@'), '\002', '^B'), value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q new file mode 100644 index 0000000000000..956a61f7bd3c1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q @@ -0,0 +1,15 @@ +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/bmjpathfilter; + +create table t1 (dt string) location '${system:test.tmp.dir}/bmjpathfilter/t1'; +Create table t2 (dt string) stored as orc; +dfs -touchz ${system:test.tmp.dir}/bmjpathfilter/t1/_SUCCESS; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +SET hive.optimize.bucketmapjoin=true; + +SELECT /*+ MAPJOIN(b) */ a.dt FROM t1 a JOIN t2 b ON (a.dt = b.dt); + +SET hive.optimize.bucketmapjoin=false; +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +dfs -rmr ${system:test.tmp.dir}/bmjpathfilter; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q index 33dd5d5cd2e0a..6bdb09ed64b07 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q @@ -9,8 +9,8 @@ sorted by (key, value) into 1 BUCKETS stored as textfile; create table table2(key string, value string) clustered by (value, key) sorted by (value, key) into 1 BUCKETS stored as textfile; -load data local inpath '../data/files/SortCol1Col2.txt' overwrite into table table1; -load data local inpath '../data/files/SortCol2Col1.txt' overwrite into table table2; +load data local inpath '../../data/files/SortCol1Col2.txt' overwrite into table table1; +load data local inpath '../../data/files/SortCol2Col1.txt' overwrite into table table2; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q index d1097e70a9880..07f6d150ea053 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q @@ -9,8 +9,8 @@ sorted by (key desc, value desc) into 1 BUCKETS stored as textfile; create table table2(key string, value string) clustered by (value, key) sorted by (value desc, key desc) into 1 BUCKETS stored as textfile; -load data local inpath '../data/files/SortCol1Col2.txt' overwrite into table table1; -load data local inpath '../data/files/SortCol2Col1.txt' overwrite into table table2; +load data local inpath '../../data/files/SortCol1Col2.txt' overwrite into table table1; +load data local inpath '../../data/files/SortCol2Col1.txt' overwrite into table table2; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q new file mode 100644 index 0000000000000..c9266a59c342a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q @@ -0,0 +1,85 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); + +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'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); + +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; +insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin_part; + +CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin; + +set hive.convert.join.bucket.mapjoin.tez = true; +explain +select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key; + +-- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table. +-- In this case the sub-query is chosen as the big table. +explain +select a.k1, a.v1, b.value +from (select sum(substr(srcbucket_mapjoin.value,5)) as v1, key as k1 from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a +join tab b on a.k1 = b.key; + +explain +select a.k1, a.v1, b.value +from (select sum(substr(tab.value,5)) as v1, key as k1 from tab_part join tab on tab_part.key = tab.key GROUP BY tab.key) a +join tab b on a.k1 = b.key; + +explain +select a.k1, a.v1, b.value +from (select sum(substr(x.value,5)) as v1, x.key as k1 from tab x join tab y on x.key = y.key GROUP BY x.key) a +join tab_part b on a.k1 = b.key; + +-- multi-way join +explain +select a.key, a.value, b.value +from tab_part a join tab b on a.key = b.key join tab c on a.key = c.key; + +explain +select a.key, a.value, c.value +from (select x.key, x.value from tab_part x join tab y on x.key = y.key) a join tab c on a.key = c.key; + +-- in this case sub-query is the small table +explain +select a.key, a.value, b.value +from (select key, sum(substr(srcbucket_mapjoin.value,5)) as value from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a +join tab_part b on a.key = b.key; + +set hive.map.aggr=false; +explain +select a.key, a.value, b.value +from (select key, sum(substr(srcbucket_mapjoin.value,5)) as value from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a +join tab_part b on a.key = b.key; + +-- join on non-bucketed column results in broadcast join. +explain +select a.key, a.value, b.value +from tab a join tab_part b on a.value = b.value; + +CREATE TABLE tab1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab1 +select key,value from srcbucket_mapjoin; + +explain +select a.key, a.value, b.value +from tab1 a join tab_part b on a.key = b.key; + +explain select a.key, b.key from tab_part a join tab_part c on a.key = c.key join tab_part b on a.value = b.value; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q new file mode 100644 index 0000000000000..a3588ec94ccee --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q @@ -0,0 +1,50 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); + +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'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); + +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; +insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin_part; + +CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin; + +set hive.convert.join.bucket.mapjoin.tez = true; + +explain select a.key, b.key from tab_part a join tab_part c on a.key = c.key join tab_part b on a.value = b.value; + +CREATE TABLE tab1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab1 +select key,value from srcbucket_mapjoin; + +explain +select a.key, a.value, b.value +from tab1 a join src b on a.key = b.key; + +explain +select a.key, b.key from (select key from tab_part where key > 1) a join (select key from tab_part where key > 2) b on a.key = b.key; + +explain +select a.key, b.key from (select key from tab_part where key > 1) a left outer join (select key from tab_part where key > 2) b on a.key = b.key; + +explain +select a.key, b.key from (select key from tab_part where key > 1) a right outer join (select key from tab_part where key > 2) b on a.key = b.key; + +explain select a.key, b.key from (select distinct key from tab) a join tab b on b.key = a.key; + +explain select a.value, b.value from (select distinct value from tab) a join tab b on b.key = a.value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q index 5b013995d5ba4..047a2a5230ae0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q @@ -1,18 +1,18 @@ -- small 1 part, 2 bucket & big 2 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q index f952f2ee6d445..d58e8448a09a8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q @@ -1,16 +1,16 @@ -- small 1 part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q index 461fbb196e758..fd80174f23119 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q @@ -1,16 +1,16 @@ -- small 2 part, 2 bucket & big 1 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q index 366da4473b36d..5d21ea5d63218 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q @@ -1,18 +1,18 @@ -- small 2 part, 4 bucket & big 1 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q index 411fed392a45e..5078072677be6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q @@ -1,13 +1,13 @@ -- small no part, 4 bucket & big no part, 2 bucket CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q index 204d1e9010d5d..0f7c72f4eaf43 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q @@ -1,16 +1,16 @@ -- small no part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q index b0bca460f303f..c528f61302743 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q @@ -1,21 +1,21 @@ -- small 2 part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q index 9533c5512f3dd..27c55a36f445b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q @@ -1,21 +1,21 @@ -- small 2 part, 2 bucket & big 2 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q index 4c7f0c9595488..d2e12e82d4a26 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q @@ -3,7 +3,7 @@ set mapred.min.split.size = 64; CREATE TABLE T1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1; CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE; @@ -26,8 +26,8 @@ EXPLAIN SELECT COUNT(1) FROM T2; SELECT COUNT(1) FROM T2; CREATE TABLE T3(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T3; -LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T3; EXPLAIN SELECT COUNT(1) FROM T3; SELECT COUNT(1) FROM T3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q index 9cdfe8e6e9f6e..8b7535dce1ee2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q @@ -1,17 +1,17 @@ CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q index 2bd8e1e2e5d98..204e75988fb54 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q @@ -25,16 +25,16 @@ select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b on a.key=b.key where b.ds="2008-04-08"; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; -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'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +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'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -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'); +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'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q index cf4222bdd781e..09c0ae2bfcbc1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q @@ -2,23 +2,23 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (key) INTO 3 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 3 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q index e10ab522a296d..d330b7718c5f2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q @@ -2,25 +2,25 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (key) INTO 4 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); set hive.optimize.bucketmapjoin=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q index 8139000f4a733..43a9de4e8edef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q @@ -2,20 +2,20 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 NOT CLUSTERED; CREATE TABLE srcbucket_mapjoin_part_3 (key INT, value STRING) PARTITIONED BY (part STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_3 CLUSTERED BY (key) INTO 2 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q index fdbadfcb04c9c..108b67a9f7644 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q @@ -1,12 +1,12 @@ CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 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'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +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'); +load data local inpath '../../data/files/srcbucket23.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'); +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'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); @@ -76,8 +76,8 @@ from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b on a.key = b.key; -- HIVE-3210 -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q index 8fda80231522e..78c23d5132503 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q @@ -1,16 +1,16 @@ CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 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'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +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'); +load data local inpath '../../data/files/srcbucket23.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'); +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'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q index c1a8f2ab7bfa0..54626e7c48ded 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q @@ -1,17 +1,17 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 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'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +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'); +load data local inpath '../../data/files/srcbucket23.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'); +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'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q index 2df49b64f3227..72cffc2da2874 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q @@ -1,22 +1,22 @@ CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 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'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); +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'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); 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'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +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'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q index 3a96c642b3e4c..a15570b77687f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q @@ -2,13 +2,13 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); set hive.optimize.bucketmapjoin=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q index 5e8daa5f8f125..f467ea6cb3ea1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q @@ -2,13 +2,13 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 3 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q index 86344a53f5a95..f1d5f581848d0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q @@ -2,14 +2,14 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; @@ -28,8 +28,8 @@ ON a.key = b.key AND a.part = '1' and b.part = '1'; ALTER TABLE srcbucket_mapjoin_part_2 DROP PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q index d7634333e2fee..ea140ddda91fe 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q @@ -3,13 +3,13 @@ CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; 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'); +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'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q index 901f0567590ec..e2c0d8c591719 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q @@ -1,12 +1,12 @@ CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; 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'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +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'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q index d66e1238bb79b..6398fff909eab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q @@ -8,21 +8,21 @@ create table test2 (key string, value string) clustered by (value) sorted by (va create table test3 (key string, value string) clustered by (key, value) sorted by (key, value) into 3 buckets; create table test4 (key string, value string) clustered by (value, key) sorted by (value, key) into 3 buckets; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test1; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test1; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test1; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test1; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test1; +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test1; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test2; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test2; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test2; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test2; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test2; +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test2; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test3; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test3; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test3; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test3; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test3; +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test3; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test4; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test4; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test4; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test4; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test4; +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test4; set hive.optimize.bucketmapjoin = true; -- should be allowed diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q index 729ffdc8687b5..b1551f2feb900 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + -- cast string floats to integer types select cast('1' as float), @@ -27,4 +29,4 @@ select cast('127' as tinyint), cast('1.0a' as int), cast('-1.-1' as int) -from src limit 1; +from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q new file mode 100644 index 0000000000000..840a84a7b3e30 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q @@ -0,0 +1,32 @@ +drop table char1; +drop table char1_1; + +create table char1 (key char(10), value char(20)); +create table char1_1 (key string, value string); + +-- load from file +load data local inpath '../../data/files/srcbucket0.txt' overwrite into table char1; +select * from char1 order by key, value limit 2; + +-- insert overwrite, from same/different length char +insert overwrite table char1 + select cast(key as char(10)), cast(value as char(15)) from src order by key, value limit 2; +select key, value from char1 order by key, value; + +-- insert overwrite, from string +insert overwrite table char1 + select key, value from src order by key, value limit 2; +select key, value from char1 order by key, value; + +-- insert string from char +insert overwrite table char1_1 + select key, value from char1 order by key, value limit 2; +select key, value from char1_1 order by key, value; + +-- respect string length +insert overwrite table char1 + select key, cast(value as char(3)) from src order by key, value limit 2; +select key, value from char1 order by key, value; + +drop table char1; +drop table char1_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q new file mode 100644 index 0000000000000..3e4900cb7c949 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q @@ -0,0 +1,36 @@ +drop table char_2; + +create table char_2 ( + key char(10), + value char(20) +); + +insert overwrite table char_2 select * from src; + +select value, sum(cast(key as int)), count(*) numrows +from src +group by value +order by value asc +limit 5; + +-- should match the query from src +select value, sum(cast(key as int)), count(*) numrows +from char_2 +group by value +order by value asc +limit 5; + +select value, sum(cast(key as int)), count(*) numrows +from src +group by value +order by value desc +limit 5; + +-- should match the query from src +select value, sum(cast(key as int)), count(*) numrows +from char_2 +group by value +order by value desc +limit 5; + +drop table char_2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q new file mode 100644 index 0000000000000..7f44d4d508537 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q @@ -0,0 +1,92 @@ + +-- Cast from char to other data types +select + cast(cast('11' as string) as tinyint), + cast(cast('11' as string) as smallint), + cast(cast('11' as string) as int), + cast(cast('11' as string) as bigint), + cast(cast('11.00' as string) as float), + cast(cast('11.00' as string) as double), + cast(cast('11.00' as string) as decimal) +from src limit 1; + +select + cast(cast('11' as char(10)) as tinyint), + cast(cast('11' as char(10)) as smallint), + cast(cast('11' as char(10)) as int), + cast(cast('11' as char(10)) as bigint), + cast(cast('11.00' as char(10)) as float), + cast(cast('11.00' as char(10)) as double), + cast(cast('11.00' as char(10)) as decimal) +from src limit 1; + +select + cast(cast('2011-01-01' as string) as date), + cast(cast('2011-01-01 01:02:03' as string) as timestamp) +from src limit 1; + +select + cast(cast('2011-01-01' as char(10)) as date), + cast(cast('2011-01-01 01:02:03' as char(30)) as timestamp) +from src limit 1; + +-- no tests from string/char to boolean, that conversion doesn't look useful +select + cast(cast('abc123' as string) as string), + cast(cast('abc123' as string) as varchar(10)), + cast(cast('abc123' as string) as char(10)) +from src limit 1; + +select + cast(cast('abc123' as char(10)) as string), + cast(cast('abc123' as char(10)) as varchar(10)), + cast(cast('abc123' as char(10)) as char(10)) +from src limit 1; + +select + cast(cast('abc123' as varchar(10)) as string), + cast(cast('abc123' as varchar(10)) as varchar(10)), + cast(cast('abc123' as varchar(10)) as char(10)) +from src limit 1; + +-- cast from other types to char +select + cast(cast(11 as tinyint) as string), + cast(cast(11 as smallint) as string), + cast(cast(11 as int) as string), + cast(cast(11 as bigint) as string), + cast(cast(11.00 as float) as string), + cast(cast(11.00 as double) as string), + cast(cast(11.00 as decimal) as string) +from src limit 1; + +select + cast(cast(11 as tinyint) as char(10)), + cast(cast(11 as smallint) as char(10)), + cast(cast(11 as int) as char(10)), + cast(cast(11 as bigint) as char(10)), + cast(cast(11.00 as float) as char(10)), + cast(cast(11.00 as double) as char(10)), + cast(cast(11.00 as decimal) as char(10)) +from src limit 1; + +select + cast(date '2011-01-01' as string), + cast(timestamp('2011-01-01 01:02:03') as string) +from src limit 1; + +select + cast(date '2011-01-01' as char(10)), + cast(timestamp('2011-01-01 01:02:03') as char(30)) +from src limit 1; + +select + cast(true as string), + cast(false as string) +from src limit 1; + +select + cast(true as char(10)), + cast(false as char(10)) +from src limit 1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q new file mode 100644 index 0000000000000..e1cfdb2bdb07a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q @@ -0,0 +1,40 @@ + +-- Should all be true +select + cast('abc' as char(10)) = cast('abc' as char(10)), + cast('abc' as char(10)) <= cast('abc' as char(10)), + cast('abc' as char(10)) >= cast('abc' as char(10)), + cast('abc' as char(10)) < cast('abd' as char(10)), + cast('abc' as char(10)) > cast('abb' as char(10)), + cast('abc' as char(10)) <> cast('abb' as char(10)) +from src limit 1; + +-- Different char lengths should still compare the same +select + cast('abc' as char(10)) = cast('abc' as char(3)), + cast('abc' as char(10)) <= cast('abc' as char(3)), + cast('abc' as char(10)) >= cast('abc' as char(3)), + cast('abc' as char(10)) < cast('abd' as char(3)), + cast('abc' as char(10)) > cast('abb' as char(3)), + cast('abc' as char(10)) <> cast('abb' as char(3)) +from src limit 1; + +-- Should work with string types as well +select + cast('abc' as char(10)) = 'abc', + cast('abc' as char(10)) <= 'abc', + cast('abc' as char(10)) >= 'abc', + cast('abc' as char(10)) < 'abd', + cast('abc' as char(10)) > 'abb', + cast('abc' as char(10)) <> 'abb' +from src limit 1; + +-- leading space is significant for char +select + cast(' abc' as char(10)) <> cast('abc' as char(10)) +from src limit 1; + +-- trailing space is not significant for char +select + cast('abc ' as char(10)) = cast('abc' as char(10)) +from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q new file mode 100644 index 0000000000000..373352dee0078 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q @@ -0,0 +1,35 @@ +drop table char_join1_ch1; +drop table char_join1_ch2; +drop table char_join1_str; + +create table char_join1_ch1 ( + c1 int, + c2 char(10) +); + +create table char_join1_ch2 ( + c1 int, + c2 char(20) +); + +create table char_join1_str ( + c1 int, + c2 string +); + +load data local inpath '../../data/files/vc1.txt' into table char_join1_ch1; +load data local inpath '../../data/files/vc1.txt' into table char_join1_ch2; +load data local inpath '../../data/files/vc1.txt' into table char_join1_str; + +-- Join char with same length char +select * from char_join1_ch1 a join char_join1_ch1 b on (a.c2 = b.c2) order by a.c1; + +-- Join char with different length char +select * from char_join1_ch1 a join char_join1_ch2 b on (a.c2 = b.c2) order by a.c1; + +-- Join char with string +select * from char_join1_ch1 a join char_join1_str b on (a.c2 = b.c2) order by a.c1; + +drop table char_join1_ch1; +drop table char_join1_ch2; +drop table char_join1_str; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q new file mode 100644 index 0000000000000..c710b6c73174b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q @@ -0,0 +1,53 @@ +drop table char_nested_1; +drop table char_nested_array; +drop table char_nested_map; +drop table char_nested_struct; +drop table char_nested_cta; +drop table char_nested_view; + +create table char_nested_1 (key int, value char(20)); +insert overwrite table char_nested_1 + select key, value from src order by key limit 1; + +-- arrays +create table char_nested_array (c1 array); +insert overwrite table char_nested_array + select array(value, value) from char_nested_1; +describe char_nested_array; +select * from char_nested_array; + +-- maps +create table char_nested_map (c1 map); +insert overwrite table char_nested_map + select map(key, value) from char_nested_1; +describe char_nested_map; +select * from char_nested_map; + +-- structs +create table char_nested_struct (c1 struct); +insert overwrite table char_nested_struct + select named_struct('a', key, + 'b', value, + 'c', cast(value as string)) + from char_nested_1; +describe char_nested_struct; +select * from char_nested_struct; + +-- nested type with create table as +create table char_nested_cta as + select * from char_nested_struct; +describe char_nested_cta; +select * from char_nested_cta; + +-- nested type with view +create table char_nested_view as + select * from char_nested_struct; +describe char_nested_view; +select * from char_nested_view; + +drop table char_nested_1; +drop table char_nested_array; +drop table char_nested_map; +drop table char_nested_struct; +drop table char_nested_cta; +drop table char_nested_view; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q new file mode 100644 index 0000000000000..4340b4de6d597 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q @@ -0,0 +1,102 @@ +drop table if exists char_serde_regex; +drop table if exists char_serde_lb; +drop table if exists char_serde_ls; +drop table if exists char_serde_c; +drop table if exists char_serde_lbc; +drop table if exists char_serde_orc; + +-- +-- RegexSerDe +-- +create table char_serde_regex ( + key char(15), + value char(20) +) +row format serde 'org.apache.hadoop.hive.serde2.RegexSerDe' +with serdeproperties ( + "input.regex" = "([^]*)([^]*)" +) +stored as textfile; + +load data local inpath '../../data/files/srcbucket0.txt' overwrite into table char_serde_regex; + +select * from char_serde_regex limit 5; +select value, count(*) from char_serde_regex group by value limit 5; + +-- +-- LazyBinary +-- +create table char_serde_lb ( + key char(15), + value char(20) +); +alter table char_serde_lb set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; + +insert overwrite table char_serde_lb + select key, value from char_serde_regex; +select * from char_serde_lb limit 5; +select value, count(*) from char_serde_lb group by value limit 5; + +-- +-- LazySimple +-- +create table char_serde_ls ( + key char(15), + value char(20) +); +alter table char_serde_ls set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; + +insert overwrite table char_serde_ls + select key, value from char_serde_lb; +select * from char_serde_ls limit 5; +select value, count(*) from char_serde_ls group by value limit 5; + +-- +-- Columnar +-- +create table char_serde_c ( + key char(15), + value char(20) +) stored as rcfile; +alter table char_serde_c set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; + +insert overwrite table char_serde_c + select key, value from char_serde_ls; +select * from char_serde_c limit 5; +select value, count(*) from char_serde_c group by value limit 5; + +-- +-- LazyBinaryColumnar +-- +create table char_serde_lbc ( + key char(15), + value char(20) +) stored as rcfile; +alter table char_serde_lbc set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; + +insert overwrite table char_serde_lbc + select key, value from char_serde_c; +select * from char_serde_lbc limit 5; +select value, count(*) from char_serde_lbc group by value limit 5; + +-- +-- ORC +-- +create table char_serde_orc ( + key char(15), + value char(20) +) stored as orc; +alter table char_serde_orc set serde 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'; + + +insert overwrite table char_serde_orc + select key, value from char_serde_lbc; +select * from char_serde_orc limit 5; +select value, count(*) from char_serde_orc group by value limit 5; + +drop table if exists char_serde_regex; +drop table if exists char_serde_lb; +drop table if exists char_serde_ls; +drop table if exists char_serde_c; +drop table if exists char_serde_lbc; +drop table if exists char_serde_orc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q new file mode 100644 index 0000000000000..629d41dca9870 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q @@ -0,0 +1,156 @@ +drop table char_udf_1; + +create table char_udf_1 (c1 string, c2 string, c3 char(10), c4 char(20)); +insert overwrite table char_udf_1 + select key, value, key, value from src where key = '238' limit 1; + +-- UDFs with char support +select + concat(c1, c2), + concat(c3, c4), + concat(c1, c2) = concat(c3, c4) +from char_udf_1 limit 1; + +select + upper(c2), + upper(c4), + upper(c2) = upper(c4) +from char_udf_1 limit 1; + +select + lower(c2), + lower(c4), + lower(c2) = lower(c4) +from char_udf_1 limit 1; + +-- Scalar UDFs +select + ascii(c2), + ascii(c4), + ascii(c2) = ascii(c4) +from char_udf_1 limit 1; + +select + concat_ws('|', c1, c2), + concat_ws('|', c3, c4), + concat_ws('|', c1, c2) = concat_ws('|', c3, c4) +from char_udf_1 limit 1; + +select + decode(encode(c2, 'US-ASCII'), 'US-ASCII'), + decode(encode(c4, 'US-ASCII'), 'US-ASCII'), + decode(encode(c2, 'US-ASCII'), 'US-ASCII') = decode(encode(c4, 'US-ASCII'), 'US-ASCII') +from char_udf_1 limit 1; + +select + instr(c2, '_'), + instr(c4, '_'), + instr(c2, '_') = instr(c4, '_') +from char_udf_1 limit 1; + +select + length(c2), + length(c4), + length(c2) = length(c4) +from char_udf_1 limit 1; + +select + locate('a', 'abcdabcd', 3), + locate(cast('a' as char(1)), cast('abcdabcd' as char(10)), 3), + locate('a', 'abcdabcd', 3) = locate(cast('a' as char(1)), cast('abcdabcd' as char(10)), 3) +from char_udf_1 limit 1; + +select + lpad(c2, 15, ' '), + lpad(c4, 15, ' '), + lpad(c2, 15, ' ') = lpad(c4, 15, ' ') +from char_udf_1 limit 1; + +select + ltrim(c2), + ltrim(c4), + ltrim(c2) = ltrim(c4) +from char_udf_1 limit 1; + +select + regexp(c2, 'val'), + regexp(c4, 'val'), + regexp(c2, 'val') = regexp(c4, 'val') +from char_udf_1 limit 1; + +select + regexp_extract(c2, 'val_([0-9]+)', 1), + regexp_extract(c4, 'val_([0-9]+)', 1), + regexp_extract(c2, 'val_([0-9]+)', 1) = regexp_extract(c4, 'val_([0-9]+)', 1) +from char_udf_1 limit 1; + +select + regexp_replace(c2, 'val', 'replaced'), + regexp_replace(c4, 'val', 'replaced'), + regexp_replace(c2, 'val', 'replaced') = regexp_replace(c4, 'val', 'replaced') +from char_udf_1 limit 1; + +select + reverse(c2), + reverse(c4), + reverse(c2) = reverse(c4) +from char_udf_1 limit 1; + +select + rpad(c2, 15, ' '), + rpad(c4, 15, ' '), + rpad(c2, 15, ' ') = rpad(c4, 15, ' ') +from char_udf_1 limit 1; + +select + rtrim(c2), + rtrim(c4), + rtrim(c2) = rtrim(c4) +from char_udf_1 limit 1; + +select + sentences('See spot run. See jane run.'), + sentences(cast('See spot run. See jane run.' as char(50))) +from char_udf_1 limit 1; + +select + split(c2, '_'), + split(c4, '_') +from char_udf_1 limit 1; + +select + str_to_map('a:1,b:2,c:3',',',':'), + str_to_map(cast('a:1,b:2,c:3' as char(20)),',',':') +from char_udf_1 limit 1; + +select + substr(c2, 1, 3), + substr(c4, 1, 3), + substr(c2, 1, 3) = substr(c4, 1, 3) +from char_udf_1 limit 1; + +select + trim(c2), + trim(c4), + trim(c2) = trim(c4) +from char_udf_1 limit 1; + + +-- Aggregate Functions +select + compute_stats(c2, 16), + compute_stats(c4, 16) +from char_udf_1; + +select + min(c2), + min(c4) +from char_udf_1; + +select + max(c2), + max(c4) +from char_udf_1; + + +drop table char_udf_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q new file mode 100644 index 0000000000000..2ce5e89c2aa9e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q @@ -0,0 +1,47 @@ +drop table char_union1_ch1; +drop table char_union1_ch2; +drop table char_union1_str; + +create table char_union1_ch1 ( + c1 int, + c2 char(10) +); + +create table char_union1_ch2 ( + c1 int, + c2 char(20) +); + +create table char_union1_str ( + c1 int, + c2 string +); + +load data local inpath '../../data/files/vc1.txt' into table char_union1_ch1; +load data local inpath '../../data/files/vc1.txt' into table char_union1_ch2; +load data local inpath '../../data/files/vc1.txt' into table char_union1_str; + +-- union char with same length char +select * from ( + select * from char_union1_ch1 + union all + select * from char_union1_ch1 limit 1 +) q1 sort by c1; + +-- union char with different length char +select * from ( + select * from char_union1_ch1 + union all + select * from char_union1_ch2 limit 1 +) q1 sort by c1; + +-- union char with string +select * from ( + select * from char_union1_ch1 + union all + select * from char_union1_str limit 1 +) q1 sort by c1; + +drop table char_union1_ch1; +drop table char_union1_ch2; +drop table char_union1_str; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q new file mode 100644 index 0000000000000..332b84087e8ef --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS char_varchar_udf; + +CREATE TABLE char_varchar_udf (c char(8), vc varchar(10)) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; +LOAD DATA LOCAL INPATH '../../data/files/char_varchar_udf.txt' INTO TABLE char_varchar_udf; + +SELECT ROUND(c, 2), ROUND(vc, 3) FROM char_varchar_udf; +SELECT AVG(c), AVG(vc), SUM(c), SUM(vc) FROM char_varchar_udf; + +DROP TABLE char_varchar_udf; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q index 3c8a309991d75..fbf8bba56e52c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q @@ -4,7 +4,7 @@ SET hive.stats.collect.scancols=true; -- This test is used for testing the ColumnAccessAnalyzer CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q index 98627776d3163..9dfe8ffbc3058 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q @@ -4,8 +4,8 @@ DROP TABLE Employee_Part; CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=2000.0); -LOAD DATA LOCAL INPATH "../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=4000.0); +LOAD DATA LOCAL INPATH "../../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=2000.0); +LOAD DATA LOCAL INPATH "../../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=4000.0); explain 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/clientpositive/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q index 72d88a67b5212..170fbc51916a0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q @@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none ( 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; +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 sourceIP, avgTimeOnSite, adRevenue; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q new file mode 100644 index 0000000000000..6be02ec8a7bbb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q @@ -0,0 +1,12 @@ + +compile `import org.apache.hadoop.hive.ql.exec.UDF \; +public class Pyth extends UDF { + public double evaluate(double a, double b){ + return Math.sqrt((a*a) + (b*b)) \; + } +} ` AS GROOVY NAMED Pyth.groovy; +CREATE TEMPORARY FUNCTION Pyth as 'Pyth'; + +SELECT Pyth(3,4) FROM src tablesample (1 rows); + +DROP TEMPORARY FUNCTION Pyth; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q index c19813602782f..fd15634f202f7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q @@ -1,7 +1,7 @@ create table tab_binary(a binary); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/binary.txt" INTO TABLE tab_binary; +LOAD DATA LOCAL INPATH "../../data/files/binary.txt" INTO TABLE tab_binary; select count(*) from tab_binary; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q index dc76f7c7d57e1..cddb53f8f64f5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q @@ -1,7 +1,7 @@ create table tab_bool(a boolean); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/bool.txt" INTO TABLE tab_bool; +LOAD DATA LOCAL INPATH "../../data/files/bool.txt" INTO TABLE tab_bool; select count(*) from tab_bool; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q new file mode 100644 index 0000000000000..77ec066dad985 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q @@ -0,0 +1,11 @@ +set hive.stats.autogather=true; + +create table tab_decimal(a decimal(10,3)); + +-- insert some data +LOAD DATA LOCAL INPATH "../../data/files/decimal.txt" INTO TABLE tab_decimal; + +select count(*) from tab_decimal; + +-- compute statistical summary of data +select compute_stats(a, 18) from tab_decimal; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q index 6c6dc47e67701..7a1e0f6295dc4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q @@ -1,7 +1,7 @@ create table tab_double(a double); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/double.txt" INTO TABLE tab_double; +LOAD DATA LOCAL INPATH "../../data/files/double.txt" INTO TABLE tab_double; select count(*) from tab_double; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q index 7d0a1584a28cc..6a2070f7808f5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q @@ -1,7 +1,7 @@ create table tab_int(a int); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/int.txt" INTO TABLE tab_int; +LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int; select count(*) from tab_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q index f146f6b95b544..0023e7f6bd818 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q @@ -1,7 +1,7 @@ create table tab_string(a string); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/string.txt" INTO TABLE tab_string; +LOAD DATA LOCAL INPATH "../../data/files/string.txt" INTO TABLE tab_string; select count(*) from tab_string; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q index ced72d60445c2..d51b80194e90a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT NAMED_STRUCT( IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1, @@ -7,7 +9,7 @@ SELECT NAMED_STRUCT( IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1, IF(ARRAY_CONTAINS(MAP_KEYS(MAP("b", "x")), "b"), "F2", "B2"), 2 ).F2 - FROM src LIMIT 1; + FROM src tablesample (1 rows); SELECT NAMED_STRUCT( IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1, @@ -17,4 +19,4 @@ SELECT NAMED_STRUCT( IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1, IF(ARRAY_CONTAINS(MAP_KEYS(MAP("b", "x")), "b"), "F2", "B2"), 2 ).F2 - FROM src LIMIT 1; + FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q index b3fd3f760eaf0..0596f965ed89c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q @@ -104,7 +104,7 @@ FROM (SELECT x.key AS key, count(1) AS cnt set hive.optimize.correlation=false; -- If the key of a GroupByOperator is the right table's key in --- a Left Outer Join, we cannot use a single MR to execute these two +-- a Left Outer Join, we cannot use a single MR to execute these two -- operators because those keys with a null value are not grouped. EXPLAIN SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) @@ -129,6 +129,29 @@ FROM (SELECT y.key AS key, count(1) AS cnt FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) GROUP BY y.key) tmp; +set hive.optimize.correlation=false; +-- If a column of the key of a GroupByOperator is the right table's key in +-- a Left Outer Join, we cannot use a single MR to execute these two +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT x.key, y.value, count(1) AS cnt +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value) +GROUP BY x.key, y.value; + +SELECT x.key, y.value, count(1) AS cnt +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value) +GROUP BY x.key, y.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT x.key, y.value, count(1) AS cnt +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value) +GROUP BY x.key, y.value; + +SELECT x.key, y.value, count(1) AS cnt +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value) +GROUP BY x.key, y.value; + set hive.optimize.correlation=false; -- If the key of a GroupByOperator is the right table's key in -- a Right Outer Join, these two operators will be executed in diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q index 70fcdfc0b4288..953d191fc6087 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q @@ -1,9 +1,9 @@ CREATE TABLE T1(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T3; set hive.auto.convert.join=false; set hive.optimize.correlation=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q index ac836c0bfdc91..287c7a389d1a8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q @@ -1,11 +1,11 @@ CREATE TABLE T1(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1; CREATE TABLE T2(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T2; CREATE TABLE T3(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/kv3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/kv3.txt' INTO TABLE T3; CREATE TABLE T4(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE T4; +LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' INTO TABLE T4; CREATE TABLE dest_co1(key INT, val STRING); CREATE TABLE dest_co2(key INT, val STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q index 0d66a5ec345e6..74ae9e428befe 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q @@ -1,5 +1,5 @@ create table abcd (a int, b int, c int, d int); -LOAD DATA LOCAL INPATH '../data/files/in4.txt' INTO TABLE abcd; +LOAD DATA LOCAL INPATH '../../data/files/in4.txt' INTO TABLE abcd; select * from abcd; set hive.map.aggr=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q new file mode 100644 index 0000000000000..ad924d345304c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q @@ -0,0 +1,30 @@ + +-- qtest_get_java_boolean should already be created during test initialization +select qtest_get_java_boolean('true'), qtest_get_java_boolean('false') from src limit 1; + +create database mydb; +create function mydb.func1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'; + +show functions mydb.func1; + +select mydb.func1('abc') from src limit 1; + +drop function mydb.func1; + +-- function should now be gone +show functions mydb.func1; + +-- To test function name resolution +create function mydb.qtest_get_java_boolean as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'; + +use default; +-- unqualified function should resolve to one in default db +select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1; + +use mydb; +-- unqualified function should resolve to one in mydb db +select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1; + +drop function mydb.qtest_get_java_boolean; + +drop database mydb cascade; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q index cb4d6578af476..13539a65f512f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q @@ -21,8 +21,8 @@ INSERT OVERWRITE TABLE table2 SELECT key, value FROM src WHERE key = 100; SELECT * FROM table1; SELECT * FROM table2; -CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test'; -CREATE EXTERNAL TABLE table5 LIKE table4 LOCATION '${system:test.src.data.dir}/files/ext_test'; +CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:hive.root}/data/files/ext_test'; +CREATE EXTERNAL TABLE table5 LIKE table4 LOCATION '${system:hive.root}/data/files/ext_test'; SELECT * FROM table4; SELECT * FROM table5; @@ -31,5 +31,5 @@ DROP TABLE table5; SELECT * FROM table4; DROP TABLE table4; -CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test'; +CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:hive.root}/data/files/ext_test'; SELECT * FROM table4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q index 4418b3430affc..483931b6fff4b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q @@ -1,6 +1,6 @@ create table src_rc_merge_test(key int, value string) stored as rcfile; -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test; +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test; set hive.exec.compress.output = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q index 2debd0d71d224..735b139719700 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q @@ -9,7 +9,7 @@ CREATE TABLE table1 ( DESCRIBE table1; DESCRIBE EXTENDED table1; -LOAD DATA LOCAL INPATH '../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table1; +LOAD DATA LOCAL INPATH '../../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table1; SELECT * from table1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q index dd5aa63e4578a..1e5d151f286ba 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q @@ -4,7 +4,7 @@ row format delimited fields terminated by '\t' collection items terminated by '\001'; -load data local inpath '../data/files/kv1.txt' +load data local inpath '../../data/files/kv1.txt' overwrite into table abc; SELECT strct, strct.a, strct.b FROM abc LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q index bb0e5b989d54d..6bc4d29358c11 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q @@ -4,7 +4,7 @@ strct struct); create table abc(mydata uniontype,struct>, strct struct); -load data local inpath '../data/files/union_input.txt' +load data local inpath '../../data/files/union_input.txt' overwrite into table abc; SELECT * FROM abc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q index 2199750f42f7e..11ba9c8afc61d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q @@ -11,3 +11,14 @@ describe formatted w; drop view v; drop view w; + + +-- HIVE-4116 Can't use views using map datatype. + +CREATE TABLE items (id INT, name STRING, info MAP); + +explain +CREATE VIEW priceview AS SELECT items.id, items.info['price'] FROM items; +CREATE VIEW priceview AS SELECT items.id, items.info['price'] FROM items; + +select * from priceview; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q new file mode 100644 index 0000000000000..fb38c947b32d4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q @@ -0,0 +1,26 @@ + +create table A as +select * from src; + +create table B as +select * from src +limit 10; + +set hive.auto.convert.join.noconditionaltask.size=100; + +explain select * from A join B; + +explain select * from B d1 join B d2 on d1.key = d2.key join A; + +explain select * from A join + (select d1.key + from B d1 join B d2 on d1.key = d2.key + where 1 = 1 group by d1.key) od1; + +explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1; + +explain select * from +(select A.key from A group by key) ss join +(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q new file mode 100644 index 0000000000000..479d57137e946 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q @@ -0,0 +1,27 @@ +create table A as +select * from src; + +create table B as +select * from src +limit 10; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000000; + +explain select * from A join B; + +explain select * from B d1 join B d2 on d1.key = d2.key join A; + +explain select * from A join + (select d1.key + from B d1 join B d2 on d1.key = d2.key + where 1 = 1 group by d1.key) od1; + +explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1; + +explain select * from +(select A.key from A group by key) ss join +(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q index e595904b412fa..71af40e7e4140 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q @@ -56,7 +56,7 @@ set hive.exec.mode.local.auto=true; create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10; create table nzhang_ctas6 (key string, `to` string); -insert overwrite table nzhang_ctas6 select key, value from src limit 10; +insert overwrite table nzhang_ctas6 select key, value from src tablesample (10 rows); create table nzhang_ctas7 as select key, `to` from nzhang_ctas6; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q new file mode 100644 index 0000000000000..ecfe74afd0273 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q @@ -0,0 +1,22 @@ +drop table ctas_char_1; +drop table ctas_char_2; +drop view ctas_char_3; + +create table ctas_char_1 (key char(10), value string); +insert overwrite table ctas_char_1 + select key, value from src sort by key, value limit 5; + +-- create table as with char column +create table ctas_char_2 as select key, value from ctas_char_1; + +-- view with char column +create view ctas_char_3 as select key, value from ctas_char_2; + +select key, value from ctas_char_1; +select * from ctas_char_2; +select * from ctas_char_3; + + +drop table ctas_char_1; +drop table ctas_char_2; +drop view ctas_char_3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q index 4961b971dbdec..f39689de03a55 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q @@ -58,11 +58,6 @@ create table nzhang_ctas6 (key string, `to` string); insert overwrite table nzhang_ctas6 select key, value from src limit 10; create table nzhang_ctas7 as select key, `to` from nzhang_ctas6; - - - - - - - - +create table nzhang_ctas8 as select 3.14BD from nzhang_ctas6 limit 1; +desc nzhang_ctas8; +drop table nzhang_ctas8; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q new file mode 100644 index 0000000000000..ca4132f7d3fb9 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q @@ -0,0 +1,28 @@ +explain +with q1 as ( select key from src where key = '5') +select * +from q1 +; + +with q1 as ( select key from src where key = '5') +select * +from q1 +; + +-- in subquery +explain +with q1 as ( select key from src where key = '5') +select * from (select key from q1) a; + +with q1 as ( select key from src where key = '5') +select * from (select key from q1) a; + +-- chaining +explain +with q1 as ( select key from q2 where key = '5'), +q2 as ( select key from src where key = '5') +select * from (select key from q1) a; + +with q1 as ( select key from q2 where key = '5'), +q2 as ( select key from src 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/clientpositive/cte_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q new file mode 100644 index 0000000000000..b49620aca144d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q @@ -0,0 +1,56 @@ + +-- union test +with q1 as (select * from src where key= '5'), +q2 as (select * from src s2 where key = '4') +select * from q1 union all select * from q2 +; + +-- insert test +create table s1 like src; +with q1 as ( select key, value from src where key = '5') +from q1 +insert overwrite table s1 +select * +; +select * from s1; +drop table s1; + +-- from style +with q1 as (select * from src where key= '5') +from q1 +select * +; + +-- ctas +create table s2 as +with q1 as ( select key from src where key = '4') +select * from q1 +; + +select * from s2; +drop table s2; + +-- view test +create view v1 as +with q1 as ( select key from src where key = '5') +select * from q1 +; + +select * from v1; + +drop view v1; + + +-- view test, name collision +create view v1 as +with q1 as ( select key from src where key = '5') +select * from q1 +; + +with q1 as ( select key from src where key = '4') +select * from v1 +; + +drop view v1; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q index ff5e86dc5a6b4..d769d05602a17 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q @@ -1,6 +1,7 @@ -ADD JAR ../build/ql/test/test-udfs.jar; CREATE TABLE src1_rot13_iof(key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.udf.Rot13InputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.udf.Rot13OutputFormat'; +DESCRIBE EXTENDED src1_rot13_iof; +SELECT * FROM src1 ORDER BY key, value; INSERT OVERWRITE TABLE src1_rot13_iof SELECT * FROM src1; -SELECT * FROM src1_rot13_iof; +SELECT * FROM src1_rot13_iof ORDER BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q index 9140a42b650e8..e3ceaccfa5ef6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q @@ -58,7 +58,7 @@ SHOW TABLES; DESCRIBE EXTENDED test_table_like; -- LOAD and SELECT -LOAD DATA LOCAL INPATH '../data/files/test.dat' +LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE test_table; SELECT * FROM test_table; @@ -146,7 +146,7 @@ CREATE TABLE db1.src(key STRING, value STRING) STORED AS TEXTFILE; -- LOAD into foreign table -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE db1.src; -- SELECT from foreign table @@ -158,7 +158,7 @@ PARTITIONED BY (ds STRING, hr STRING) STORED AS TEXTFILE; -- LOAD data into Partitioned foreign table -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE db1.srcpart PARTITION (ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q index 4e17c7ad5ac10..1371273245215 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q @@ -8,13 +8,14 @@ CREATE DATABASE db5; SHOW DATABASES; USE db5; +set hive.stats.dbclass=fs; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp; dfs -rmr ${system:test.tmp.dir}/dbcascade; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade; -- add a table, index and view CREATE TABLE temp_tbl (id INT, name STRING); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE temp_tbl; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE temp_tbl; CREATE VIEW temp_tbl_view AS SELECT * FROM temp_tbl; CREATE INDEX idx1 ON TABLE temp_tbl(id) AS 'COMPACT' with DEFERRED REBUILD; ALTER INDEX idx1 ON temp_tbl REBUILD; @@ -23,15 +24,15 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp_tbl2; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp_tbl2_idx2; -- add a table, index and view with a different storage location CREATE TABLE temp_tbl2 (id INT, name STRING) LOCATION 'file:${system:test.tmp.dir}/dbcascade/temp_tbl2'; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' into table temp_tbl2; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' into table temp_tbl2; CREATE VIEW temp_tbl2_view AS SELECT * FROM temp_tbl2; CREATE INDEX idx2 ON TABLE temp_tbl2(id) AS 'COMPACT' with DEFERRED REBUILD LOCATION 'file:${system:test.tmp.dir}/dbcascade/temp_tbl2_idx2'; ALTER INDEX idx2 ON temp_tbl2 REBUILD; -- add a partitioned table, index and view CREATE TABLE part_tab (id INT, name STRING) PARTITIONED BY (ds string); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2009-04-09'); CREATE INDEX idx3 ON TABLE part_tab(id) AS 'COMPACT' with DEFERRED REBUILD; ALTER INDEX idx3 ON part_tab PARTITION (ds='2008-04-09') REBUILD; ALTER INDEX idx3 ON part_tab PARTITION (ds='2009-04-09') REBUILD; @@ -41,8 +42,8 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab2_idx4; -- add a partitioned table, index and view with a different storage location CREATE TABLE part_tab2 (id INT, name STRING) PARTITIONED BY (ds string) LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab2'; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2009-04-09'); CREATE INDEX idx4 ON TABLE part_tab2(id) AS 'COMPACT' with DEFERRED REBUILD LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab2_idx4'; ALTER INDEX idx4 ON part_tab2 PARTITION (ds='2008-04-09') REBUILD; @@ -56,8 +57,8 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab3_idx5; CREATE TABLE part_tab3 (id INT, name STRING) PARTITIONED BY (ds string) LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3'; ALTER TABLE part_tab3 ADD PARTITION (ds='2007-04-09') LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3_p1'; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2009-04-09'); CREATE INDEX idx5 ON TABLE part_tab3(id) AS 'COMPACT' with DEFERRED REBUILD LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3_idx5'; ALTER INDEX idx5 ON part_tab3 PARTITION (ds='2008-04-09') REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q index a2322fc1e45a0..7d89ac9268f9b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q @@ -1,21 +1,23 @@ +set hive.fetch.task.conversion=more; + drop table date_1; create table date_1 (d date); insert overwrite table date_1 - select cast('2011-01-01' as date) from src limit 1; + select cast('2011-01-01' as date) from src tablesample (1 rows); select * from date_1 limit 1; select d, count(d) from date_1 group by d; insert overwrite table date_1 - select date '2011-01-01' from src limit 1; + select date '2011-01-01' from src tablesample (1 rows); select * from date_1 limit 1; select d, count(d) from date_1 group by d; insert overwrite table date_1 - select cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src limit 1; + select cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src tablesample (1 rows); select * from date_1 limit 1; select d, count(d) from date_1 group by d; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q index 0821e012ba26b..c5346c87dd25c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q @@ -8,7 +8,7 @@ create table date_2 ( FL_NUM int ); -LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_2; +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_2; select fl_date, fl_num from date_2 order by fl_date asc, fl_num desc; select fl_date, fl_num from date_2 order by fl_date desc, fl_num asc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q index be251484459b6..383fb4e9904da 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q @@ -7,7 +7,7 @@ create table date_3 ( alter table date_3 add columns (c2 date); insert overwrite table date_3 - select 1, cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src limit 1; + select 1, cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src tablesample (1 rows); select * from date_3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q index 4801a79b8795b..c840089f2efeb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + drop table date_4; create table date_4 (d date); @@ -5,7 +7,7 @@ alter table date_4 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -- Test date literal syntax insert overwrite table date_4 - select date '2011-01-01' from src limit 1; + select date '2011-01-01' from src tablesample (1 rows); select d, date '2011-01-01' from date_4 limit 1; drop table date_4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q index bdcb6c1b6e447..86c7362e295f2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + -- Comparisons against same value select cast('2011-05-06' as date) > cast('2011-05-06' as date) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q index a5844b76e9ef8..34bb8c8990a6b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q @@ -8,7 +8,7 @@ create table date_join1 ( FL_NUM int ); -LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE date_join1; +LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE date_join1; -- Note that there are 2 rows with date 2000-11-28, so we should expect 4 rows with that date in the join results select t1.fl_num, t1.fl_date, t2.fl_num, t2.fl_date diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q index ffc06d270d53a..24b48206802ae 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q @@ -22,7 +22,7 @@ with serdeproperties ( ) stored as textfile; -load data local inpath '../data/files/flights_tiny.txt.1' overwrite into table date_serde_regex; +load data local inpath '../../data/files/flights_tiny.txt.1' overwrite into table date_serde_regex; select * from date_serde_regex; select fl_date, count(*) from date_serde_regex group by fl_date; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q index 9696320a85fcd..c55b9f9147ada 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q @@ -17,7 +17,7 @@ create table date_udf_flight ( ARR_DELAY float, FL_NUM int ); -LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight; +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight; -- Test UDFs with date input select unix_timestamp(d), year(d), month(d), day(d), dayofmonth(d), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q new file mode 100644 index 0000000000000..6612fe8babdf8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q @@ -0,0 +1,12 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.txn.testing=true; + +create table T1(key string, val string) stored as textfile; + +set hive.txn.testing=true; +alter table T1 compact 'major'; + +alter table T1 compact 'minor'; + +drop table T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q new file mode 100644 index 0000000000000..599cad9afc611 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q @@ -0,0 +1,14 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.txn.testing=true; + +create table T1(key string, val string) partitioned by (ds string) stored as textfile; + +alter table T1 add partition (ds = 'today'); +alter table T1 add partition (ds = 'yesterday'); + +alter table T1 partition (ds = 'today') compact 'major'; + +alter table T1 partition (ds = 'yesterday') compact 'minor'; + +drop table T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q new file mode 100644 index 0000000000000..871d292a59ce3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q @@ -0,0 +1,15 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.txn.testing=true; + +create database D1; + +use D1; + +create table T1(key string, val string) stored as textfile; + +alter table T1 compact 'major'; + +alter table T1 compact 'minor'; + +drop table T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q new file mode 100644 index 0000000000000..3126bd6e5460e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q @@ -0,0 +1,59 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create database D1; + +alter database D1 set dbproperties('test'='yesthisis'); + +drop database D1; + +create table T1(key string, val string) stored as textfile; + +create table T2 like T1; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T3 as select * from T1; + +create table T4 (key char(10), val decimal(5,2), b int) + partitioned by (ds string) + clustered by (b) into 10 buckets + stored as orc; + +alter table T3 rename to newT3; + +alter table T2 set tblproperties ('test'='thisisatest'); + +alter table T2 set serde 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'; +alter table T2 set serdeproperties ('test'='thisisatest'); + +alter table T2 clustered by (key) into 32 buckets; + +alter table T4 add partition (ds='today'); + +alter table T4 partition (ds='today') rename to partition(ds='yesterday'); + +alter table T4 drop partition (ds='yesterday'); + +alter table T4 add partition (ds='tomorrow'); + +create table T5 (a string, b int); +alter table T5 set fileformat orc; + +create table T7 (a string, b int); +alter table T7 set location 'file:///tmp'; + +alter table T2 touch; +alter table T4 touch partition (ds='tomorrow'); + +create view V1 as select key from T1; +alter view V1 set tblproperties ('test'='thisisatest'); +drop view V1; + + + +drop table T1; +drop table T2; +drop table newT3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q new file mode 100644 index 0000000000000..970069aca6766 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q @@ -0,0 +1,17 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) stored as textfile; + +insert into table T2 select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q new file mode 100644 index 0000000000000..00942e5357892 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q @@ -0,0 +1,17 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) stored as textfile; + +insert overwrite table T2 select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q new file mode 100644 index 0000000000000..75b642b5492cc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q @@ -0,0 +1,21 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) partitioned by (pval string) stored as textfile; + +insert into table T2 partition (pval = '1') select * from T1; + +select * from T2; + +insert overwrite table T2 partition (pval = '1') select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q new file mode 100644 index 0000000000000..57eb4424ea5cc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q @@ -0,0 +1,19 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string) partitioned by (val string) stored as textfile; + +insert overwrite table T2 partition (val) select key, val from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q new file mode 100644 index 0000000000000..d22b98fd7df2d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q @@ -0,0 +1,24 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create database foo; + +use foo; + +create table T1(key string, val string) partitioned by (ds string) stored as textfile; + +alter table T1 add partition (ds='today'); + +create view V1 as select key from T1; + +show tables; + +describe T1; + +drop view V1; + +drop table T1; + +show databases; + +drop database foo; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q new file mode 100644 index 0000000000000..7c71fdd9d230f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q @@ -0,0 +1,11 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.txn.testing=true; + +show locks; + +show locks extended; + +show locks default; + +show transactions; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q index 6c689e188a908..f52b1923eb067 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q @@ -1,18 +1,22 @@ -drop table decimal_1; +set hive.fetch.task.conversion=more; + +drop table if exists decimal_1; -create table decimal_1 (t decimal); +create table decimal_1 (t decimal(4,2), u decimal(5), v decimal); alter table decimal_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; +desc decimal_1; + insert overwrite table decimal_1 - select cast('17.29' as decimal) from src limit 1; -select cast(t as boolean) from decimal_1 limit 1; -select cast(t as tinyint) from decimal_1 limit 1; -select cast(t as smallint) from decimal_1 limit 1; -select cast(t as int) from decimal_1 limit 1; -select cast(t as bigint) from decimal_1 limit 1; -select cast(t as float) from decimal_1 limit 1; -select cast(t as double) from decimal_1 limit 1; -select cast(t as string) from decimal_1 limit 1; -select cast(t as timestamp) from decimal_1 limit 1; + select cast('17.29' as decimal(4,2)), 3.1415926BD, 3115926.54321BD from src tablesample (1 rows); +select cast(t as boolean) from decimal_1; +select cast(t as tinyint) from decimal_1; +select cast(t as smallint) from decimal_1; +select cast(t as int) from decimal_1; +select cast(t as bigint) from decimal_1; +select cast(t as float) from decimal_1; +select cast(t as double) from decimal_1; +select cast(t as string) from decimal_1; +select cast(t as timestamp) from decimal_1; drop table decimal_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q index 4890618a0dc32..2c4d919079423 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q @@ -1,40 +1,42 @@ +set hive.fetch.task.conversion=more; + drop table decimal_2; -create table decimal_2 (t decimal); +create table decimal_2 (t decimal(18,9)); alter table decimal_2 set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; insert overwrite table decimal_2 - select cast('17.29' as decimal) from src limit 1; + select cast('17.29' as decimal(4,2)) from src tablesample (1 rows); -select cast(t as boolean) from decimal_2 limit 1; -select cast(t as tinyint) from decimal_2 limit 1; -select cast(t as smallint) from decimal_2 limit 1; -select cast(t as int) from decimal_2 limit 1; -select cast(t as bigint) from decimal_2 limit 1; -select cast(t as float) from decimal_2 limit 1; -select cast(t as double) from decimal_2 limit 1; -select cast(t as string) from decimal_2 limit 1; +select cast(t as boolean) from decimal_2; +select cast(t as tinyint) from decimal_2; +select cast(t as smallint) from decimal_2; +select cast(t as int) from decimal_2; +select cast(t as bigint) from decimal_2; +select cast(t as float) from decimal_2; +select cast(t as double) from decimal_2; +select cast(t as string) from decimal_2; insert overwrite table decimal_2 - select cast('3404045.5044003' as decimal) from src limit 1; + select cast('3404045.5044003' as decimal(18,9)) from src tablesample (1 rows); -select cast(t as boolean) from decimal_2 limit 1; -select cast(t as tinyint) from decimal_2 limit 1; -select cast(t as smallint) from decimal_2 limit 1; -select cast(t as int) from decimal_2 limit 1; -select cast(t as bigint) from decimal_2 limit 1; -select cast(t as float) from decimal_2 limit 1; -select cast(t as double) from decimal_2 limit 1; -select cast(t as string) from decimal_2 limit 1; +select cast(t as boolean) from decimal_2; +select cast(t as tinyint) from decimal_2; +select cast(t as smallint) from decimal_2; +select cast(t as int) from decimal_2; +select cast(t as bigint) from decimal_2; +select cast(t as float) from decimal_2; +select cast(t as double) from decimal_2; +select cast(t as string) from decimal_2; -select cast(3.14 as decimal) from decimal_2 limit 1; -select cast(cast(3.14 as float) as decimal) from decimal_2 limit 1; -select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal) from decimal_2 limit 1; -select cast(true as decimal) from decimal_2 limit 1; -select cast(3Y as decimal) from decimal_2 limit 1; -select cast(3S as decimal) from decimal_2 limit 1; -select cast(cast(3 as int) as decimal) from decimal_2 limit 1; -select cast(3L as decimal) from decimal_2 limit 1; -select cast(0.99999999999999999999 as decimal) from decimal_2 limit 1; -select cast('0.99999999999999999999' as decimal) from decimal_2 limit 1; +select cast(3.14 as decimal(4,2)) from decimal_2; +select cast(cast(3.14 as float) as decimal(4,2)) from decimal_2; +select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) from decimal_2; +select cast(true as decimal) from decimal_2; +select cast(3Y as decimal) from decimal_2; +select cast(3S as decimal) from decimal_2; +select cast(cast(3 as int) as decimal) from decimal_2; +select cast(3L as decimal) from decimal_2; +select cast(0.99999999999999999999 as decimal(20,19)) from decimal_2; +select cast('0.99999999999999999999' as decimal(20,20)) from decimal_2; drop table decimal_2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q index 28211e3f14cc1..e4fba06fea1a9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS DECIMAL_3; -CREATE TABLE DECIMAL_3(key decimal, value int) +CREATE TABLE DECIMAL_3(key decimal(38,18), value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_3; +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_3; SELECT * FROM DECIMAL_3 ORDER BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q index e8a89c131cd85..699ba3cb4f558 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q @@ -1,15 +1,15 @@ DROP TABLE IF EXISTS DECIMAL_4_1; DROP TABLE IF EXISTS DECIMAL_4_2; -CREATE TABLE DECIMAL_4_1(key decimal, value int) +CREATE TABLE DECIMAL_4_1(key decimal(35,25), value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -CREATE TABLE DECIMAL_4_2(key decimal, value decimal) +CREATE TABLE DECIMAL_4_2(key decimal(35,25), value decimal(35,25)) STORED AS ORC; -LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_4_1; +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_4_1; INSERT OVERWRITE TABLE DECIMAL_4_2 SELECT key, key * 3 FROM DECIMAL_4_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q new file mode 100644 index 0000000000000..70e5db0f70182 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS DECIMAL_5; + +CREATE TABLE DECIMAL_5(key decimal(10,5), value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_5; + +SELECT key FROM DECIMAL_5 ORDER BY key; + +SELECT DISTINCT key FROM DECIMAL_5 ORDER BY key; + +SELECT cast(key as decimal) FROM DECIMAL_5; + +SELECT cast(key as decimal(6,3)) FROM DECIMAL_5; + +DROP TABLE DECIMAL_5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q new file mode 100644 index 0000000000000..b58e224256122 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q @@ -0,0 +1,27 @@ +DROP TABLE IF EXISTS DECIMAL_6_1; +DROP TABLE IF EXISTS DECIMAL_6_2; +DROP TABLE IF EXISTS DECIMAL_6_3; + +CREATE TABLE DECIMAL_6_1(key decimal(10,5), value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +CREATE TABLE DECIMAL_6_2(key decimal(17,4), value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/kv9.txt' INTO TABLE DECIMAL_6_1; +LOAD DATA LOCAL INPATH '../../data/files/kv9.txt' INTO TABLE DECIMAL_6_2; + +SELECT T.key from ( + SELECT key, value from DECIMAL_6_1 + UNION ALL + SELECT key, value from DECIMAL_6_2 +) T order by T.key; + +CREATE TABLE DECIMAL_6_3 AS SELECT key + 5.5 AS k, value * 11 AS v from DECIMAL_6_1 ORDER BY v; + +desc DECIMAL_6_3; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q index 589fc6597dfa4..86c14d9351ac1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q @@ -1,6 +1,6 @@ -- HIVE-5292 Join on decimal columns fails -create table src_dec (key decimal, value string); -load data local inpath '../data/files/kv1.txt' into table src_dec; +create table src_dec (key decimal(3,0), value string); +load data local inpath '../../data/files/kv1.txt' into table src_dec; select * from src_dec a join src_dec b on a.key=b.key+450; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q index 403c2be3fbc10..739352f9ef1e5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS DECIMAL_PRECISION; -CREATE TABLE DECIMAL_PRECISION(dec decimal) +CREATE TABLE DECIMAL_PRECISION(dec decimal(20,10)) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv8.txt' INTO TABLE DECIMAL_PRECISION; +LOAD DATA LOCAL INPATH '../../data/files/kv8.txt' INTO TABLE DECIMAL_PRECISION; SELECT * FROM DECIMAL_PRECISION ORDER BY dec; @@ -15,13 +15,14 @@ SELECT dec, dec / 9 FROM DECIMAL_PRECISION ORDER BY dec; SELECT dec, dec / 27 FROM DECIMAL_PRECISION ORDER BY dec; SELECT dec, dec * dec FROM DECIMAL_PRECISION ORDER BY dec; +EXPLAIN SELECT avg(dec), sum(dec) FROM DECIMAL_PRECISION; SELECT avg(dec), sum(dec) FROM DECIMAL_PRECISION; -SELECT dec * cast('123456789012345678901234567890.123456789' as decimal) FROM DECIMAL_PRECISION LIMIT 1; -SELECT * from DECIMAL_PRECISION WHERE dec > cast('123456789012345678901234567890.123456789' as decimal) LIMIT 1; -SELECT dec * 123456789012345678901234567890.123456789 FROM DECIMAL_PRECISION LIMIT 1; +SELECT dec * cast('12345678901234567890.12345678' as decimal(38,18)) FROM DECIMAL_PRECISION LIMIT 1; +SELECT * from DECIMAL_PRECISION WHERE dec > cast('1234567890123456789012345678.12345678' as decimal(38,18)) LIMIT 1; +SELECT dec * 12345678901234567890.12345678 FROM DECIMAL_PRECISION LIMIT 1; -SELECT MIN(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION; -SELECT COUNT(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION; +SELECT MIN(cast('12345678901234567890.12345678' as decimal(38,18))) FROM DECIMAL_PRECISION; +SELECT COUNT(cast('12345678901234567890.12345678' as decimal(38,18))) FROM DECIMAL_PRECISION; DROP TABLE DECIMAL_PRECISION; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q index 3556807705ae5..cf3a86cd4d78f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q @@ -8,7 +8,7 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_TEXT; +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_TEXT; SELECT * FROM DECIMAL_TEXT ORDER BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q index b5ff088d1613a..0c9f1b86a9e97 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DROP TABLE IF EXISTS DECIMAL_UDF; -CREATE TABLE DECIMAL_UDF (key decimal, value int) +CREATE TABLE DECIMAL_UDF (key decimal(20,10), value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_UDF; +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_UDF; -- addition EXPLAIN SELECT key + key FROM DECIMAL_UDF; @@ -70,8 +72,8 @@ EXPLAIN SELECT abs(key) FROM DECIMAL_UDF; SELECT abs(key) FROM DECIMAL_UDF; -- avg -EXPLAIN SELECT value, sum(key) / count(key), avg(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value; -SELECT value, sum(key) / count(key), avg(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value; +EXPLAIN SELECT value, sum(key) / count(key), avg(key), sum(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value; +SELECT value, sum(key) / count(key), avg(key), sum(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value; -- negative EXPLAIN SELECT -key FROM DECIMAL_UDF; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q index 112ac57c3be5b..14d508c07dd3d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q @@ -3,7 +3,7 @@ row format delimited fields terminated by '\t' lines terminated by '\n' stored as textfile; -LOAD DATA LOCAL INPATH '../data/files/in7.txt' INTO TABLE impressions; +LOAD DATA LOCAL INPATH '../../data/files/in7.txt' INTO TABLE impressions; select * from impressions; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q new file mode 100644 index 0000000000000..89e49311fa48d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q @@ -0,0 +1,7 @@ +create table t1 (a int, b string) partitioned by (c int, d string); +describe t1; + +set hive.display.partition.cols.separately=false; +describe t1; + +set hive.display.partition.cols.separately=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q index 6ea4156b3493e..81a5b3a6e6c82 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q @@ -1,9 +1,9 @@ set hive.fileformat.check = false; create table kv_fileformat_check_txt (key string, value string) stored as textfile; -load data local inpath '../data/files/kv1.seq' overwrite into table kv_fileformat_check_txt; +load data local inpath '../../data/files/kv1.seq' overwrite into table kv_fileformat_check_txt; create table kv_fileformat_check_seq (key string, value string) stored as sequencefile; -load data local inpath '../data/files/kv1.txt' overwrite into table kv_fileformat_check_seq; +load data local inpath '../../data/files/kv1.txt' overwrite into table kv_fileformat_check_seq; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q index 2a1e7276cacfe..991b930d54ca4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q @@ -1,7 +1,9 @@ +set hive.fetch.task.conversion=more; + SET hive.metastore.disallow.incompatible.col.type.changes=false; 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; +INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src tablesample (1 rows); SELECT * from test_table123 WHERE ds="foo1"; -- This should now work as hive.metastore.disallow.incompatible.col.type.changes is false ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q new file mode 100644 index 0000000000000..725183380b9ad --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q @@ -0,0 +1,20 @@ +set hive.stats.autogather=true; + +set hive.compute.query.using.stats=true; +create table t1 (a string, b string); + +insert into table t1 select * from src; + +analyze table t1 compute statistics for columns a,b; + +explain +select count(distinct b) from t1 group by a; + +explain +select distinct(b) from t1; + +explain +select a, count(*) from t1 group by a; + +drop table t1; +set hive.compute.query.using.stats = false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q index 798aa6d51a185..54e6a35b5adfe 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q @@ -6,7 +6,7 @@ alter table ptestfilter add partition (c=1, d=2); alter table ptestFilter add partition (c=2, d=1); alter table ptestfilter add partition (c=2, d=2); alter table ptestfilter add partition (c=3, d=1); -alter table ptestfilter add partition (c=3, d=2); +alter table ptestfilter add partition (c=30, d=2); show partitions ptestfilter; alter table ptestfilter drop partition (c=1, d=1); @@ -15,6 +15,9 @@ show partitions ptestfilter; alter table ptestfilter drop partition (c=2); show partitions ptestfilter; +alter table ptestfilter drop partition (c<4); +show partitions ptestfilter; + drop table ptestfilter; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q new file mode 100644 index 0000000000000..797a27c23b01e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q @@ -0,0 +1,8 @@ +set hive.lock.numretries=1; +set hive.lock.sleep.between.retries=1; +set hive.support.concurrency=true; +set hive.lock.manager=org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager; + +drop table if exists drop_with_concurrency_1; +create table drop_with_concurrency_1 (c1 int); +drop table drop_with_concurrency_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q index 397a2200a8972..699e58effcac0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q @@ -1,19 +1,19 @@ -create table dynamic_part_table(intcol int) partitioned by (partcol1 int, partcol2 int); +create table dynamic_part_table(intcol string) partitioned by (partcol1 string, partcol2 string); set hive.exec.dynamic.partition.mode=nonstrict; -insert into table dynamic_part_table partition(partcol1, partcol2) select 1, 1, 1 from src where key=150; +insert into table dynamic_part_table partition(partcol1, partcol2) select '1', '1', '1' from src where key=150; -insert into table dynamic_part_table partition(partcol1, partcol2) select 1, NULL, 1 from src where key=150; +insert into table dynamic_part_table partition(partcol1, partcol2) select '1', NULL, '1' from src where key=150; -insert into table dynamic_part_table partition(partcol1, partcol2) select 1, 1, NULL from src where key=150; +insert into table dynamic_part_table partition(partcol1, partcol2) select '1', '1', NULL from src where key=150; -insert into table dynamic_part_table partition(partcol1, partcol2) select 1, NULL, NULL from src where key=150; +insert into table dynamic_part_table partition(partcol1, partcol2) select '1', NULL, NULL from src where key=150; -explain extended select intcol from dynamic_part_table where partcol1=1 and partcol2=1; +explain extended select intcol from dynamic_part_table where partcol1='1' and partcol2='1'; set hive.exec.dynamic.partition.mode=strict; -explain extended select intcol from dynamic_part_table where partcol1=1 and partcol2=1; +explain extended select intcol from dynamic_part_table where partcol1='1' and partcol2='1'; -explain extended select intcol from dynamic_part_table where (partcol1=1 and partcol2=1)or (partcol1=1 and partcol2='__HIVE_DEFAULT_PARTITION__'); +explain extended select intcol from dynamic_part_table where (partcol1='1' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q new file mode 100644 index 0000000000000..5f1a5ce809e0c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q @@ -0,0 +1,161 @@ +set hive.optimize.sort.dynamic.partition=true; +set hive.exec.dynamic.partition=true; +set hive.exec.max.dynamic.partitions=1000; +set hive.exec.max.dynamic.partitions.pernode=1000; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.vectorized.execution.enabled=true; +set hive.enforce.bucketing=false; +set hive.enforce.sorting=false; + +create table over1k( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal(4,2), + bin binary) + row format delimited + fields terminated by '|'; + +load data local inpath '../../data/files/over1k' into table over1k; + +create table over1k_orc like over1k; +alter table over1k_orc set fileformat orc; +insert overwrite table over1k_orc select * from over1k; + +create table over1k_part_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint) stored as orc; + +create table over1k_part_limit_orc like over1k_part_orc; +alter table over1k_part_limit_orc set fileformat orc; + +create table over1k_part_buck_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) into 4 buckets stored as orc; + +create table over1k_part_buck_sort_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 4 buckets stored as orc; + +-- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization +explain insert overwrite table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si; +explain insert overwrite table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10; +explain insert overwrite table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +explain insert overwrite table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +insert overwrite table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si; +insert overwrite table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10; +insert overwrite table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +insert overwrite table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; + +-- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization +explain insert into table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si; +explain insert into table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10; +explain insert into table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +explain insert into table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +insert into table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si; +insert into table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10; +insert into table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +insert into table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +desc formatted over1k_part_orc partition(ds="foo",t=27); +desc formatted over1k_part_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_limit_orc partition(ds="foo",t=27); +desc formatted over1k_part_limit_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_buck_orc partition(t=27); +desc formatted over1k_part_buck_orc partition(t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_buck_sort_orc partition(t=27); +desc formatted over1k_part_buck_sort_orc partition(t="__HIVE_DEFAULT_PARTITION__"); + +select count(*) from over1k_part_orc; +select count(*) from over1k_part_limit_orc; +select count(*) from over1k_part_buck_orc; +select count(*) from over1k_part_buck_sort_orc; + +-- tests for HIVE-6883 +create table over1k_part2_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint); + +set hive.optimize.sort.dynamic.partition=false; +explain insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i; +set hive.optimize.sort.dynamic.partition=true; +explain insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i; + +set hive.optimize.sort.dynamic.partition=false; +insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i; + +desc formatted over1k_part2_orc partition(ds="foo",t=27); +desc formatted over1k_part2_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part2_orc; +select count(*) from over1k_part2_orc; + +set hive.optimize.sort.dynamic.partition=true; +insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i; + +desc formatted over1k_part2_orc partition(ds="foo",t=27); +desc formatted over1k_part2_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part2_orc; +select count(*) from over1k_part2_orc; + +-- hadoop-1 does not honor number of reducers in local mode. There is always only 1 reducer irrespective of the number of buckets. +-- Hence all records go to one bucket and all other buckets will be empty. Similar to HIVE-6867. However, hadoop-2 honors number +-- of reducers and records are spread across all reducers. To avoid this inconsistency we will make number of buckets to 1 for this test. +create table over1k_part_buck_sort2_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 1 buckets; + +set hive.optimize.sort.dynamic.partition=false; +explain insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +set hive.optimize.sort.dynamic.partition=true; +explain insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +set hive.optimize.sort.dynamic.partition=false; +insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +desc formatted over1k_part_buck_sort2_orc partition(t=27); +desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part_buck_sort2_orc; +select count(*) from over1k_part_buck_sort2_orc; + +set hive.optimize.sort.dynamic.partition=true; +insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +desc formatted over1k_part_buck_sort2_orc partition(t=27); +desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part_buck_sort2_orc; +select count(*) from over1k_part_buck_sort2_orc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q new file mode 100644 index 0000000000000..52b5d1e0c1d8a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q @@ -0,0 +1,155 @@ +set hive.optimize.sort.dynamic.partition=true; +set hive.exec.dynamic.partition=true; +set hive.exec.max.dynamic.partitions=1000; +set hive.exec.max.dynamic.partitions.pernode=1000; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.enforce.bucketing=false; +set hive.enforce.sorting=false; + +create table over1k( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal(4,2), + bin binary) + row format delimited + fields terminated by '|'; + +load data local inpath '../../data/files/over1k' into table over1k; + +create table over1k_part( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint); + +create table over1k_part_limit like over1k_part; + +create table over1k_part_buck( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) into 4 buckets; + +create table over1k_part_buck_sort( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 4 buckets; + +-- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization +explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27; +explain insert overwrite table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10; +explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27; +explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27; +insert overwrite table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10; +insert overwrite table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27; +insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; + +-- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization +explain insert into table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27; +explain insert into table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10; +explain insert into table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27; +explain insert into table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +insert into table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27; +insert into table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10; +insert into table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27; +insert into table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +desc formatted over1k_part partition(ds="foo",t=27); +desc formatted over1k_part partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_limit partition(ds="foo",t=27); +desc formatted over1k_part_limit partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_buck partition(t=27); +desc formatted over1k_part_buck partition(t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_buck_sort partition(t=27); +desc formatted over1k_part_buck_sort partition(t="__HIVE_DEFAULT_PARTITION__"); + +select count(*) from over1k_part; +select count(*) from over1k_part_limit; +select count(*) from over1k_part_buck; +select count(*) from over1k_part_buck_sort; + +-- tests for HIVE-6883 +create table over1k_part2( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint); + +set hive.optimize.sort.dynamic.partition=false; +explain insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i; +set hive.optimize.sort.dynamic.partition=true; +explain insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i; + +set hive.optimize.sort.dynamic.partition=false; +insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i; + +desc formatted over1k_part2 partition(ds="foo",t=27); +desc formatted over1k_part2 partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part2; +select count(*) from over1k_part2; + +set hive.optimize.sort.dynamic.partition=true; +insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i; + +desc formatted over1k_part2 partition(ds="foo",t=27); +desc formatted over1k_part2 partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part2; +select count(*) from over1k_part2; + +-- hadoop-1 does not honor number of reducers in local mode. There is always only 1 reducer irrespective of the number of buckets. +-- Hence all records go to one bucket and all other buckets will be empty. Similar to HIVE-6867. However, hadoop-2 honors number +-- of reducers and records are spread across all reducers. To avoid this inconsistency we will make number of buckets to 1 for this test. +create table over1k_part_buck_sort2( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 1 buckets; + +set hive.optimize.sort.dynamic.partition=false; +explain insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27; +set hive.optimize.sort.dynamic.partition=true; +explain insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +set hive.optimize.sort.dynamic.partition=false; +insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +desc formatted over1k_part_buck_sort2 partition(t=27); +desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part_buck_sort2; +select count(*) from over1k_part_buck_sort2; + +set hive.optimize.sort.dynamic.partition=true; +insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +desc formatted over1k_part_buck_sort2 partition(t=27); +desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part_buck_sort2; +select count(*) from over1k_part_buck_sort2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q index a7f4cf79c980f..d29a7a8f105d6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q @@ -8,7 +8,7 @@ DROP TABLE escape1; DROP TABLE escape_raw; CREATE TABLE escape_raw (s STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/escapetest.txt' INTO TABLE escape_raw; +LOAD DATA LOCAL INPATH '../../data/files/escapetest.txt' INTO TABLE escape_raw; SELECT count(*) from escape_raw; SELECT * from escape_raw; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q index 473cbf8c94e4e..24601343b1474 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q @@ -10,7 +10,7 @@ DROP TABLE IF EXISTS escape2; DROP TABLE IF EXISTS escape_raw; CREATE TABLE escape_raw (s STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/escapetest.txt' INTO TABLE escape_raw; +LOAD DATA LOCAL INPATH '../../data/files/escapetest.txt' INTO TABLE escape_raw; SELECT count(*) from escape_raw; SELECT * from escape_raw; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q index 6e8bf8ebc1fa2..4be6e3f6d876b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q @@ -3,7 +3,7 @@ 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; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q index 27b335a3d7844..f346ddeec4963 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q @@ -3,7 +3,7 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN 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_test2 ADD PARTITION (ds='2013-04-05', hr='1'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q index 7b9060d420a1e..7c076cebe8ad4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q @@ -3,8 +3,9 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN 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_test1 ADD PARTITION (ds='2014-01-03', hr='1'); +ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1'); +ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='2'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q index 7fa96b629a0fa..8288bbfd86acc 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q @@ -1,3 +1,6 @@ +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.test.mode=true; set hive.test.mode.prefix=; set hive.test.mode.nosamplelist=exim_department,exim_employee; @@ -5,8 +8,8 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/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; @@ -16,7 +19,7 @@ use importer; import from 'ql/test/data/exports/exim_department'; describe extended exim_department; show table extended like exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q index 9920e778d18a6..1e2eed803a01d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q @@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -17,7 +17,7 @@ use importer; import from 'ql/test/data/exports/exim_department'; describe extended exim_department; show table extended like exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q index 4017c83aa3dd4..474a5a4a5bb5f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q @@ -7,8 +7,8 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -18,7 +18,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q index 21138f0263418..dbd2c6bf5c4c5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q @@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id") 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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -20,7 +20,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q index 5f6bdee83cc90..47d949aa36d9b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q @@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -21,6 +21,6 @@ import from 'ql/test/data/exports/exim_department'; describe extended exim_department; select * from exim_department; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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/clientpositive/exim_04_all_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q index 69c6faa30a07c..b2567fb270326 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -26,7 +26,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q index cdc02fa25c169..82df69874b47a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q @@ -19,8 +19,8 @@ alter table exim_employee set fileformat outputformat "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"; alter table exim_employee add partition (emp_country='in', emp_state='ka'); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -32,7 +32,7 @@ describe extended exim_employee; describe extended exim_employee partition (emp_country='in', emp_state='tn'); describe extended exim_employee partition (emp_country='in', emp_state='ka'); show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q index 50a59463b1870..a2c977356c84c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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 partition (emp_state="ka") to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -26,7 +26,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q index 5136090929fc5..3a612964222eb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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 partition (emp_country="in",emp_state="ka") to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -26,7 +26,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q index 5b9d4ddc03c59..8c774d5a8ba84 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -28,12 +28,12 @@ create table exim_employee ( emp_id int comment "employee id") 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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="al"); import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/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/clientpositive/exim_08_nonpart_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q index 173f1569c501a..8a1d9454763c7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q @@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee,exim_imported_dept 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -17,12 +17,12 @@ create table exim_department ( dep_id int comment "department id") partitioned by (emp_org string) stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department partition (emp_org="hr"); +load data local inpath "../../data/files/test.dat" into table exim_department partition (emp_org="hr"); import table exim_imported_dept from 'ql/test/data/exports/exim_department'; describe extended exim_imported_dept; select * from exim_imported_dept; drop table exim_imported_dept; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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/clientpositive/exim_09_part_spec_nonoverlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q index 178b76674d862..53fc2936bb2f3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -27,14 +27,14 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); import table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/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/clientpositive/exim_10_external_managed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q index 413f2aa4762f5..54859eed196cd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q @@ -2,18 +2,18 @@ set hive.test.mode=true; set hive.test.mode.prefix=; set hive.test.mode.nosamplelist=exim_department,exim_employee; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +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 external table exim_department ( dep_id int comment "department id") stored as textfile location 'ql/test/data/tablestore/exim_department' tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; create database importer; use importer; @@ -22,6 +22,6 @@ import from 'ql/test/data/exports/exim_department'; describe extended exim_department; select * from exim_department; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/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/clientpositive/exim_11_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q index f3b2896a322a5..4fc39dcff0620 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q @@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -17,7 +17,7 @@ use importer; import external table exim_department from 'ql/test/data/exports/exim_department'; describe extended exim_department; select * from exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q index 37d063432e16a..e4d50ffe5b8f0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q @@ -5,24 +5,24 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/ql/test/data/exports/exim_department/temp; +dfs -rmr ${system:test.tmp.dir}/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} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department/temp; +dfs -rmr ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department; import external table exim_department from 'ql/test/data/exports/exim_department' location 'ql/test/data/tablestore/exim_department'; describe extended exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr ${system:test.tmp.dir}/ql/test/data/exports/exim_department; select * from exim_department; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q index fb5058b840bae..909d23794be9d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q @@ -5,24 +5,24 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; import table exim_department from 'ql/test/data/exports/exim_department' location 'ql/test/data/tablestore/exim_department'; describe extended exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q index 031b6bda6cece..dbb5fd93435a0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q @@ -5,17 +5,17 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +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 @@ -24,9 +24,9 @@ create table exim_department ( dep_id int comment "department id") import table exim_department from 'ql/test/data/exports/exim_department' location 'ql/test/data/tablestore/exim_department'; describe extended exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q index ff088c70d7ffb..989dd6cf56547 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q @@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; create external table exim_employee ( emp_id int comment "employee id") comment "employee table" @@ -32,17 +32,17 @@ create external table exim_employee ( emp_id int comment "employee id") stored as textfile location 'ql/test/data/tablestore/exim_employee' tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); import external table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q index 6f4ee7a01c256..7eec358850e29 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q @@ -7,26 +7,26 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore2/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore2/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore2/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore2/exim_employee; create external table exim_employee ( emp_id int comment "employee id") comment "employee table" @@ -39,11 +39,11 @@ import table exim_employee partition (emp_country="us", emp_state="tn") location 'ql/test/data/tablestore/exim_employee'; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/tablestore2/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore2/exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q index 56ec152948aed..20cd7e0513c7b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q @@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; create table exim_employee ( emp_id int comment "employee id") comment "employee table" @@ -39,9 +39,9 @@ alter table exim_employee add partition (emp_country="us", emp_state="ap") show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); show table extended like exim_employee partition (emp_country="us", emp_state="ap"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q index 7aa1297dc7d02..a300b1dbf1285 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -29,7 +29,7 @@ describe extended exim_employee; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); select * from exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q index cb9f8efc087da..a821c75d70cca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q @@ -7,20 +7,20 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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/test2.dat" +load data local inpath "../../data/files/test2.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; import external table exim_employee from 'ql/test/data/exports/exim_employee' @@ -29,9 +29,9 @@ describe extended exim_employee; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="in", emp_state="tn"); show table extended like exim_employee partition (emp_country="in", emp_state="ka"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q index bdbd19df70a21..be1216453bc18 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q @@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; import external table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee' @@ -32,9 +32,9 @@ import external table exim_employee partition (emp_country="us", emp_state="tn") describe extended exim_employee; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q index eb44961a9b7ca..000904aa6634f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q @@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id") 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" +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" +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" +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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; import table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee' @@ -32,9 +32,9 @@ import table exim_employee partition (emp_country="us", emp_state="tn") describe extended exim_employee; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q index 822ed70a38c0c..293a011cb20f6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q @@ -2,13 +2,13 @@ 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; +load data local inpath "../../data/files/test.dat" into table exim_department; set hive.security.authorization.enabled=true; grant Select on table exim_department to user hive_test_user; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/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'; set hive.security.authorization.enabled=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q index 440d08d2dc7b5..03714ab17dcbf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q @@ -3,9 +3,9 @@ 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} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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; @@ -22,5 +22,5 @@ set hive.security.authorization.enabled=false; select * from exim_department; drop table exim_department; drop database importer; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q index 30fc343dd8f9c..cb6af0efbca62 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q @@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id") 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" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +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; @@ -29,6 +29,6 @@ import from 'ql/test/data/exports/exim_employee'; set hive.security.authorization.enabled=false; select * from exim_employee; -dfs -rmr ../build/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/clientpositive/exim_24_import_nonexist_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q index 2dc5af6ce4774..8934c47372384 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q @@ -3,9 +3,9 @@ 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} ../build/ql/test/data/exports/exim_department/test; -dfs -rmr ../build/ql/test/data/exports/exim_department; +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/test; +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; @@ -20,5 +20,5 @@ set hive.security.authorization.enabled=false; select * from exim_department; drop table exim_department; drop database importer; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q new file mode 100644 index 0000000000000..f58c9f948d62b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q @@ -0,0 +1,22 @@ +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) partitioned by (emp_country string); +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in"); + +dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/exim_employee/emp_country=in/_logs; +dfs -touchz ${system:test.warehouse.dir}/exim_employee/emp_country=in/_logs/job.xml; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; + +import from 'ql/test/data/exports/exim_employee'; +describe formatted exim_employee; +select * from exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; +drop table exim_employee; +drop database importer; +use default; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q new file mode 100644 index 0000000000000..ca2da354bd367 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q @@ -0,0 +1,98 @@ +-- query from auto_sortmerge_join_9.q + +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; + +set hive.auto.convert.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.auto.convert.sortmerge.join=true; +set hive.auto.convert.sortmerge.join.to.mapjoin=false; + +set hive.explain.dependency.append.tasktype=true; + +-- default behavior + +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select 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 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; + +set hive.stageid.rearrange=IDONLY; + +-- changes id only + +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select 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 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; + +set hive.stageid.rearrange=TRAVERSE; + +-- assign ids in traverse order + +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select 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 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; + +set hive.stageid.rearrange=EXECUTION; + +-- assign ids in execution order + +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select 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 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/clientpositive/external_table_with_space_in_location_path.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q new file mode 100644 index 0000000000000..ad070464f9c4e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q @@ -0,0 +1,23 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test/; + +dfs -copyFromLocal ../../data/files/ext_test_space hdfs:///tmp/test/ext_test_space; + +CREATE EXTERNAL TABLE spacetest (id int, message string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/ext_test_space/folder+with space'; + +SELECT * FROM spacetest; + +SELECT count(*) FROM spacetest; + +DROP TABLE spacetest; + +CREATE EXTERNAL TABLE spacetestpartition (id int, message string) PARTITIONED BY (day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; + +ALTER TABLE spacetestpartition ADD PARTITION (day=10) LOCATION 'hdfs:///tmp/test/ext_test_space/folder+with space'; + +SELECT * FROM spacetestpartition; + +SELECT count(*) FROM spacetestpartition; + +DROP TABLE spacetestpartition; + +dfs -rmr hdfs:///tmp/test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q new file mode 100644 index 0000000000000..8b65c7896d590 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q @@ -0,0 +1,39 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test/; + +dfs -copyFromLocal ../../data/files/header_footer_table_1 hdfs:///tmp/test/header_footer_table_1; + +dfs -copyFromLocal ../../data/files/header_footer_table_2 hdfs:///tmp/test/header_footer_table_2; + +dfs -copyFromLocal ../../data/files/header_footer_table_3 hdfs:///tmp/test/header_footer_table_3; + +CREATE EXTERNAL TABLE header_footer_table_1 (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/header_footer_table_1' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2"); + +SELECT * FROM header_footer_table_1; + +SELECT * FROM header_footer_table_1 WHERE id < 50; + +CREATE EXTERNAL TABLE header_footer_table_2 (name string, message string, id int) PARTITIONED BY (year int, month int, day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2"); + +ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=1) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/01'; + +ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=2) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/02'; + +ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=3) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/03'; + +SELECT * FROM header_footer_table_2; + +SELECT * FROM header_footer_table_2 WHERE id < 50; + +CREATE EXTERNAL TABLE emptytable (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/header_footer_table_3' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2"); + +SELECT * FROM emptytable; + +SELECT * FROM emptytable WHERE id < 50; + +DROP TABLE header_footer_table_1; + +DROP TABLE header_footer_table_2; + +DROP TABLE emptytable; + +dfs -rmr hdfs:///tmp/test; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q index f8d855b25948a..7f4258f7bc15b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q @@ -12,11 +12,11 @@ create table T3 (c0 bigint, c1 bigint, c2 int) partitioned by (ds string); create table T4 (c0 bigint, c1 string, c2 string, c3 string, c4 string, c5 string, c6 string, c7 string, c8 string, c9 string, c10 string, c11 string, c12 string, c13 string, c14 string, c15 string, c16 string, c17 string, c18 string, c19 string, c20 string, c21 string, c22 string, c23 string, c24 string, c25 string, c26 string, c27 string, c28 string, c29 string, c30 string, c31 string, c32 string, c33 string, c34 string, c35 string, c36 string, c37 string, c38 string, c39 string, c40 string, c41 string, c42 string, c43 string, c44 string, c45 string, c46 string, c47 string, c48 string, c49 string, c50 string, c51 string, c52 string, c53 string, c54 string, c55 string, c56 string, c57 string, c58 string, c59 string, c60 string, c61 string, c62 string, c63 string, c64 string, c65 string, c66 string, c67 bigint, c68 string, c69 string, c70 bigint, c71 bigint, c72 bigint, c73 string, c74 string, c75 string, c76 string, c77 string, c78 string, c79 string, c80 string, c81 bigint, c82 bigint, c83 bigint) partitioned by (ds string); -insert overwrite table T1 partition (ds='2010-04-17') select '5', '1', '1', '1', 0, 0,4 from src limit 1; +insert overwrite table T1 partition (ds='2010-04-17') select '5', '1', '1', '1', 0, 0,4 from src tablesample (1 rows); -insert overwrite table T2 partition(ds='2010-04-17') select '5','name', NULL, '2', 'kavin',NULL, '9', 'c', '8', '0', '0', '7', '1','2', '0', '3','2', NULL, '1', NULL, '3','2','0','0','5','10' from src limit 1; +insert overwrite table T2 partition(ds='2010-04-17') select '5','name', NULL, '2', 'kavin',NULL, '9', 'c', '8', '0', '0', '7', '1','2', '0', '3','2', NULL, '1', NULL, '3','2','0','0','5','10' from src tablesample (1 rows); -insert overwrite table T3 partition (ds='2010-04-17') select 4,5,0 from src limit 1; +insert overwrite table T3 partition (ds='2010-04-17') select 4,5,0 from src tablesample (1 rows); insert overwrite table T4 partition(ds='2010-04-17') select 4,'1','1','8','4','5','1','0','9','U','2','2', '0','2','1','1','J','C','A','U', '2','s', '2',NULL, NULL, NULL,NULL, NULL, NULL,'1','j', 'S', '6',NULL,'1', '2', 'J', 'g', '1', 'e', '2', '1', '2', 'U', 'P', 'p', '3', '0', '0', '0', '1', '1', '1', '0', '0', '0', '6', '2', 'j',NULL, NULL, NULL,NULL,NULL, NULL, '5',NULL, 'j', 'j', 2, 2, 1, '2', '2', '1', '1', '1', '1', '1', '1', 1, 1, 32,NULL from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q new file mode 100644 index 0000000000000..69d543f472347 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q @@ -0,0 +1,21 @@ +set hive.exec.dynamic.partition.mode=nonstrict; + +create table partint(key string, value string) partitioned by (ds string, hr int); +insert overwrite table partint partition(ds, hr) select * from srcpart where ds = '2008-04-08'; + +explain select key, value, hr from partint where hr < 11; +select key, value, hr from partint where hr < 11; + +explain select key, value, hr from partint where hr <= 12 and hr > 11; +select key, value, hr from partint where hr <= 12 and hr > 11; + +explain select key, value, hr from partint where hr between 11 and 12; +select key, value, hr from partint where hr between 11 and 12; + +explain select key, value, hr from partint where hr not between 12 and 14; +select key, value, hr from partint where hr not between 12 and 14; + +explain select key, value, hr from partint where hr < 13; +select key, value, hr from partint where hr < 13; + +drop table partint; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q index b76cf34120701..c8a08af054579 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q @@ -8,9 +8,9 @@ drop table gl_src_part1; create table gl_src1 (key int, value string) stored as textfile; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1; @@ -49,10 +49,10 @@ select key from gl_src2 ORDER BY key ASC limit 10; -- partition create table gl_src_part1 (key int, value string) partitioned by (p string) stored as textfile; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); select key from gl_src_part1 where p like '1%' ORDER BY key ASC limit 10; select key from gl_src_part1 where p='11' ORDER BY key ASC limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q index db38d43fe4354..7750cb90b5ca5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q @@ -6,7 +6,7 @@ CREATE TABLE dest1(key INT, val1 INT, val2 INT); CREATE TABLE dest2(key INT, val1 INT, val2 INT); CREATE TABLE INPUT(key INT, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE INPUT; +LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' INTO TABLE INPUT; EXPLAIN FROM INPUT diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q new file mode 100644 index 0000000000000..6e9aee1d11d58 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q @@ -0,0 +1,13 @@ +set hive.map.aggr=false; + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT COUNT(src.key), COUNT(DISTINCT value) GROUP BY src.key; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT COUNT(src.key), COUNT(DISTINCT value) GROUP BY src.key; + +SELECT dest1.* FROM dest1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q index 1b6891e33a37e..55133332a8662 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q @@ -5,6 +5,6 @@ CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE; EXPLAIN FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key LIMIT 5; -FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key LIMIT 5; +FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key ORDER BY src.key LIMIT 5; SELECT dest1.* FROM dest1 ORDER BY dest1.key ASC , dest1.value ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q index 82cff36422e62..dde37dfd47145 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q @@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) G FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q index 874995888b447..f346cb7e90147 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q @@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) G FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q index 1b10f1e024819..c587b5f658f68 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q @@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; -SELECT dest_g1.* FROM dest_g1; +SELECT dest_g1.* FROM dest_g1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q index 4bc263c77f1b4..30499248cac15 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q @@ -1,7 +1,7 @@ set mapred.reduce.tasks=31; EXPLAIN -SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 5; +SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5; -SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 5; +SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q index c3cf598fb1c90..794ec758e9edb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q @@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(s 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))) GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q index 25e6789b63ef2..55d1a34b3c921 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q @@ -11,4 +11,15 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(s 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; +SELECT dest1.* FROM dest1 ORDER BY key; + +-- HIVE-5560 when group by key is used in distinct funtion, invalid result are returned + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.key,1,1)), 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.key,1,1)), 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 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q index c3c82d51749fe..6d7cb61e2d44a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q @@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr 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))) GROUP BY substr(src.key,1,1); -SELECT dest_g2.* FROM dest_g2; +SELECT dest_g2.* FROM dest_g2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q index b80c271afe5dc..b2450c9ea04e1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q @@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr 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; +SELECT dest_g2.* FROM dest_g2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q index 99c2d2d5a1784..a1ebf90aadfea 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q @@ -12,5 +12,5 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key, FROM src INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q index be60785d87ea5..e96568b398d87 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q @@ -16,5 +16,5 @@ SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key; -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q index fbf761c3aea55..ced122fae3f50 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q @@ -11,6 +11,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); FROM src INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q index ac79a286055d5..0d3727b052858 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q @@ -11,6 +11,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); FROM src INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q index 2c99d362ffff7..466c13222f29f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q @@ -12,6 +12,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); FROM src INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q index b1457d9349aee..2b8c5db41ea92 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q @@ -18,5 +18,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q index 481b3cd084f16..5895ed4599849 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q @@ -17,5 +17,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q index a34ac8f64b117..ee6d7bf83084e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q @@ -17,5 +17,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q index 94a3dcf7ec3f2..8c2308e5d75c3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q @@ -18,5 +18,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q index 802aea244da9e..e673cc61622c8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q @@ -14,8 +14,8 @@ INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY S INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10; FROM SRC -INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10 -INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10; +INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key ORDER BY SRC.key limit 10 +INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key ORDER BY SRC.key limit 10; SELECT DEST1.* FROM DEST1 ORDER BY key ASC, value ASC; SELECT DEST2.* FROM DEST2 ORDER BY key ASC, value ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q index 62b6ff5ddebb6..0252e993363aa 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q @@ -14,6 +14,6 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q index 846fd01017172..b5e1f63a45257 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q @@ -14,6 +14,6 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q index 20c8bef34223a..da85504ca18c6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q @@ -15,5 +15,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q index 7e97f75cecc92..2e3eddcb1f866 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q @@ -1,7 +1,7 @@ set hive.map.aggr.hash.percentmemory = 0.3; set hive.mapred.local.mem = 384; -add file ../data/scripts/dumpdata_script.py; +add file ../../data/scripts/dumpdata_script.py; select count(distinct subq.key) from (FROM src MAP src.key USING 'python dumpdata_script.py' AS key WHERE src.key = 10) subq; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q index 46e1f00d0f224..099beb4319e09 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q @@ -3,7 +3,7 @@ set hive.groupby.skewindata=false; CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; EXPLAIN SELECT key, val, count(1) FROM T1 GROUP BY key, val with cube; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q index bced21f9e494a..de4a7c3cb5e54 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q @@ -1,6 +1,6 @@ CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; SELECT key, val, GROUPING__ID from T1 group by key, val with cube; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q index ffc627c82eaff..f451f17834502 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q @@ -1,6 +1,6 @@ CREATE TABLE T1(key INT, value INT) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/groupby_groupingid.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/groupby_groupingid.txt' INTO TABLE T1; set hive.groupby.skewindata = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q index 4fba7338f1d0a..804dfb36cf2c6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q @@ -1,6 +1,6 @@ CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1; SELECT * FROM T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q index 9f2286cc9be31..30f1b420cc7c2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q @@ -2,7 +2,7 @@ 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; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1; -- Since 4 grouping sets would be generated for the query below, an additional MR job should be created EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q index 9a00d0a7aa77f..707737798dd6a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q @@ -4,8 +4,8 @@ -- additional MR job is created for processing the grouping sets. CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets2.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets2.txt' INTO TABLE T1; set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; set hive.new.job.grouping.set.cardinality = 30; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q index 25f1fcd793a74..ff83185d819c5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q @@ -4,7 +4,7 @@ set hive.merge.mapredfiles = false; CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1; -- This tests that cubes and rollups work fine inside sub-queries. EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q index fb0c5913fa07f..d94bd81f84f3b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q @@ -4,7 +4,7 @@ set hive.merge.mapredfiles = false; CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1; -- This tests that cubes and rollups work fine where the source is a sub-query EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q index f0a8b72b0c83e..4a199365cf968 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q @@ -17,4 +17,4 @@ SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(s WHERE src.ds = '2008-04-08' GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q index b863344485d9f..cb3ee82918611 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q @@ -17,4 +17,4 @@ SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(s WHERE src.ds = '2008-04-08' GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q new file mode 100644 index 0000000000000..663e33b4c7ad8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q @@ -0,0 +1,61 @@ + + +set hive.map.aggr=false; +set hive.groupby.skewindata=false; +explain select key, count(*) from src b group by b.key; +explain select b.key, count(*) from src b group by key; + +set hive.map.aggr=false; +set hive.groupby.skewindata=true; +explain select key, count(*) from src b group by b.key; +explain select b.key, count(*) from src b group by key; + +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +explain select key, count(*) from src b group by b.key; +explain select b.key, count(*) from src b group by key; + +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +explain select key, count(*) from src b group by b.key; +explain select b.key, count(*) from src b group by key; + +-- windowing after group by +select key, count(*), rank() over(order by count(*)) +from src b +where key < '12' +group by b.key +order by b.key; + +-- having after group by +select key, count(*) +from src b +group by b.key +having key < '12' +order by b.key; + +-- having and windowing +select key, count(*), rank() over(order by count(*)) +from src b +group by b.key +having key < '12' +order by b.key +; + +explain +select key, count(*), rank() over(order by count(*)) +from src b +group by b.key +having key < '12' +; + +-- order by +select key +from src t +where key < '12' +group by t.key +order by t.key; + +-- cluster by +EXPLAIN +SELECT x.key, x.value as key FROM SRC x CLUSTER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q index f79b0c472ebb1..ee8038c7d9f5a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q @@ -3,7 +3,7 @@ set hive.groupby.skewindata=false; CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; EXPLAIN SELECT key, val, count(1) FROM T1 GROUP BY key, val with rollup; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q index 911a11ae8990f..7401a9ca1d9bd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q index 31b4ec5c74dc2..700a8af91548e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q index 103c57a123576..2ef8447935a66 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q index e43da3c93225f..3c959e381f220 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key, val) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q index bef5e5d2d547a..dd05238f1ccc4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (val) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; @@ -30,7 +30,7 @@ DROP TABLE T1; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (val, key) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; @@ -52,7 +52,7 @@ DROP TABLE T1; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (val) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q index cf076e8125612..aa09aec34b233 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q @@ -17,7 +17,7 @@ SELECT key, count(1) FROM T1 where ds = '1' GROUP BY key; SELECT * FROM outputTbl1 ORDER BY key; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='2'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='2'); -- The plan should not be converted to a map-side group since no partition is being accessed EXPLAIN EXTENDED diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q index c2d42154e516c..99337859fb00b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string) CLUSTERED BY (val) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q index 121804e60a9da..f53295e4b2435 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q index 1c3d1cdcc265d..296336d0f9f1a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q index 068c26a9c2b4b..db0faa04da0ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q @@ -7,7 +7,7 @@ set hive.groupby.skewindata=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q index 8efa05e2544d4..4ec138e51a806 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q @@ -7,7 +7,7 @@ set hive.map.groupby.sorted.testmode=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q new file mode 100644 index 0000000000000..cb147c5feab24 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q @@ -0,0 +1,13 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/; +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/exported_table/; +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/exported_table/data/; + +dfs -copyFromLocal ../../data/files/exported_table/_metadata hdfs:///tmp/test_import_exported_table/exported_table; +dfs -copyFromLocal ../../data/files/exported_table/data/data hdfs:///tmp/test_import_exported_table/exported_table/data; + +IMPORT FROM '/tmp/test_import_exported_table/exported_table'; +DESCRIBE j1_41; +SELECT * from j1_41; + +dfs -rmr hdfs:///tmp/test_import_exported_table; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q index 33a1fc581ed13..03d77f1f19b01 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q @@ -1,15 +1,18 @@ +set hive.stats.dbclass=fs; SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; create table foobar(key int, value string) PARTITIONED BY (ds string, hr string); alter table foobar add partition (ds='2008-04-08',hr='12'); -CREATE INDEX srcpart_auth_index ON TABLE foobar(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_AUTH_index ON TABLE foobar(key) as 'BITMAP' WITH DEFERRED REBUILD; +SHOW INDEXES ON foobar; + grant select on table foobar to user hive_test_user; -grant select on table default__foobar_srcpart_auth_index__ to user hive_test_user; -grant update on table default__foobar_srcpart_auth_index__ to user hive_test_user; -grant create on table default__foobar_srcpart_auth_index__ to user hive_test_user; +grant select on table default__foobar_srcpart_auth_indeX__ to user hive_test_user; +grant update on table default__foobar_srcpart_auth_indEx__ to user hive_test_user; +grant create on table default__foobar_srcpart_auth_inDex__ to user hive_test_user; set hive.security.authorization.enabled=true; -ALTER INDEX srcpart_auth_index ON foobar PARTITION (ds='2008-04-08',hr='12') REBUILD; +ALTER INDEX srcpart_auth_INDEX ON foobar PARTITION (ds='2008-04-08',hr='12') REBUILD; set hive.security.authorization.enabled=false; DROP INDEX srcpart_auth_index on foobar; DROP TABLE foobar; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q index cb8a1d6293f8d..77733aac02686 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q @@ -3,6 +3,7 @@ -- without indexing SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; @@ -25,4 +26,4 @@ SET hive.optimize.index.filter.compact.minsize=0; EXPLAIN SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key; SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key; -DROP INDEX src_index on src; \ No newline at end of file +DROP INDEX src_index on src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q index cb32162d40a06..41f4a40823e4d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q @@ -3,6 +3,7 @@ -- Create temp, and populate it with some values in src. CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE; +set hive.stats.dbclass=fs; -- Build an index on temp. CREATE INDEX temp_index ON TABLE temp(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX temp_index ON temp REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q index 790e6c223f746..2967bd60d8bc6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test automatic use of index on different file formats CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; @@ -16,4 +17,4 @@ SET hive.optimize.index.filter.compact.minsize=0; EXPLAIN SELECT key, value FROM src WHERE key=86 ORDER BY key; SELECT key, value FROM src WHERE key=86 ORDER BY key; -DROP INDEX src_index on src; \ No newline at end of file +DROP INDEX src_index on src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q index 2bf8481f1d6e2..a672e06e79332 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q @@ -4,6 +4,7 @@ EXPLAIN SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q index 808a04cc360a6..d78e0fd58a074 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q @@ -4,6 +4,7 @@ EXPLAIN SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q index 06e97fa76bc97..f0a91b4b8a592 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- With multiple indexes, make sure we choose which to use in a consistent order CREATE INDEX src_key_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; @@ -13,4 +14,4 @@ EXPLAIN SELECT key, value FROM src WHERE key=86 ORDER BY key; SELECT key, value FROM src WHERE key=86 ORDER BY key; DROP INDEX src_key_index ON src; -DROP INDEX src_val_index ON src; \ No newline at end of file +DROP INDEX src_val_index ON src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q index 5013d29e732c4..70166b36c5f3c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test automatic use of index on table with partitions CREATE INDEX src_part_index ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_part_index ON srcpart REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q index 0984a4a21ba17..1d9efbbc6d738 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q @@ -3,6 +3,7 @@ EXPLAIN SELECT a.key, b.key FROM src a JOIN src b ON (a.value = b.value) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; SELECT a.key, b.key FROM src a JOIN src b ON (a.value = b.value) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q index d8f3eda1813c8..acd4194b0e7a4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test cases where the index should not be used automatically CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q index f9deb2883960c..673c835fb9084 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; DROP INDEX srcpart_index_proj on srcpart; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q index 55633d9225fa7..adec8f1b3bfb7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; EXPLAIN CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q index bd15a21fa4200..1ffa6eeebbfb3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; EXPLAIN CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q index 0d2c811459dcb..e7a093c118ba0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q @@ -1,3 +1,6 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; + EXPLAIN CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; EXPLAIN @@ -17,10 +20,10 @@ SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ WHERE key = 0) a JOIN - (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ - WHERE value = "val_0") b - ON - a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname; INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q index 672ce29f1bb47..56cd44dd5b323 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q @@ -1,3 +1,6 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; + -- try the query without indexing, with manual indexing, and with automatic indexing -- without indexing SELECT key, value FROM src WHERE key=0 AND value = "val_0" ORDER BY key; @@ -21,12 +24,12 @@ SELECT * FROM default__src_src2_index__ ORDER BY value; EXPLAIN SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ - WHERE key = 0) a - JOIN - (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ - WHERE value = "val_0") b - ON - a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname; INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q index 90d7987594bac..3b310cee4cb29 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test automatic use of index on table with partitions CREATE INDEX src_part_index ON TABLE srcpart(key) as 'BITMAP' WITH DEFERRED REBUILD; ALTER INDEX src_part_index ON srcpart REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q index 2f5e5d4fb231b..32ecfb9db8a0e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; SET hive.exec.compress.result=true; CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q index 054df51c32180..26a351ea31858 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; CREATE TABLE srcpart_rc (key int, value string) PARTITIONED BY (ds string, hr int) STORED AS RCFILE; INSERT OVERWRITE TABLE srcpart_rc 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/clientpositive/index_compact.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q index a936f1127f690..98cbec147db51 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; DROP INDEX srcpart_index_proj on srcpart; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q index 837033be403a9..97276f488e742 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; EXPLAIN CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q index 56119ac11f32f..1eb3f5c3dbcbf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; CREATE TABLE srcpart_rc (key int, value string) PARTITIONED BY (ds string, hr int) STORED AS RCFILE; INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11; @@ -42,4 +43,4 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; SELECT key, value FROM srcpart_rc WHERE key=100 ORDER BY key; DROP INDEX srcpart_rc_index on srcpart_rc; -DROP TABLE srcpart_rc; \ No newline at end of file +DROP TABLE srcpart_rc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q index f3fcb4af3d5e6..599b4ac1149ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; CREATE TABLE src_index_test_rc (key int, value string) STORED AS RCFILE; INSERT OVERWRITE TABLE src_index_test_rc SELECT * FROM src; @@ -16,4 +17,4 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; SELECT key, value FROM src_index_test_rc WHERE key=100 ORDER BY key; DROP INDEX src_index on src_index_test_rc; -DROP TABLE src_index_test_rc; \ No newline at end of file +DROP TABLE src_index_test_rc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q index d0d9a32adcba1..e72b27c781a80 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q @@ -1,6 +1,6 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; SET hive.default.fileformat=TextFile; - +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; @@ -129,4 +129,4 @@ SELECT * FROM src WHERE key >= '9'; SET hive.exec.post.hooks=; -DROP INDEX src_index ON src; \ No newline at end of file +DROP INDEX src_index ON src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q index 84ed3cc932e8c..963b8f74e5f07 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q @@ -1,4 +1,5 @@ SET hive.exec.compress.result=true; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q index 062821e870dce..ef020b63d5a72 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; drop index src_index_2 on src; drop index src_index_3 on src; drop index src_index_4 on src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q index a6fe16ba3b809..20186a7400494 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- Want to ensure we can build and use indices on tables stored with SerDes -- Build the (Avro backed) table CREATE TABLE doctors @@ -31,7 +32,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE doctors; -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors; -- Create and build an index CREATE INDEX doctors_index ON TABLE doctors(number) AS 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q index 82e15b97b6ac4..ecab2b7f6b332 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test that stale indexes are not used CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q index e7cfeff31f6dc..a93ccf7f95fc3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- Test if index is actually being used. -- Create temp, and populate it with some values in src. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q index 119994e91b056..728b8cc4a9497 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q @@ -47,12 +47,12 @@ CREATE TABLE srcpart_merge_dp LIKE srcpart; CREATE TABLE srcpart_merge_dp_rc LIKE srcpart; ALTER TABLE srcpart_merge_dp_rc SET FILEFORMAT RCFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); -LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=12); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=12); INSERT OVERWRITE TABLE srcpart_merge_dp_rc PARTITION (ds = '2008-04-08', hr) SELECT key, value, hr FROM srcpart_merge_dp WHERE ds = '2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q index a039dc5a2a040..ce5ed8419dd44 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q @@ -1,7 +1,7 @@ DROP TABLE infertypes; CREATE TABLE infertypes(ti TINYINT, si SMALLINT, i INT, bi BIGINT, fl FLOAT, db DOUBLE, str STRING); -LOAD DATA LOCAL INPATH '../data/files/infer_const_type.txt' OVERWRITE INTO TABLE infertypes; +LOAD DATA LOCAL INPATH '../../data/files/infer_const_type.txt' OVERWRITE INTO TABLE infertypes; SELECT * FROM infertypes; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q index 40fbc84a981f1..620e9dcfae6cf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q @@ -7,15 +7,15 @@ 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 WHERE src.key >= 200 and src.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300; 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 WHERE src.key >= 200 and src.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300; SELECT dest1.* FROM dest1; SELECT dest2.* FROM dest2; SELECT dest3.* FROM dest3; -dfs -cat ../build/ql/test/data/warehouse/dest4.out/*; +dfs -cat ${system:test.warehouse.dir}/dest4.out/*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q index 82e6d81426efc..4990d0ba1cd54 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q @@ -1,6 +1,6 @@ -- TestSerDe is a user defined serde where the default delimiter is Ctrl-B DROP TABLE INPUT16; -ADD JAR ../data/files/TestSerDe.jar; +ADD 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 INPUT16(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1_cb.txt' INTO TABLE INPUT16; +LOAD DATA LOCAL INPATH '../../data/files/kv1_cb.txt' INTO TABLE INPUT16; SELECT INPUT16.VALUE, INPUT16.KEY FROM INPUT16; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q index 5dab4103d8dda..9272a92c8102e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q @@ -4,8 +4,8 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -- the user is overwriting it with ctrlC DROP TABLE INPUT16_CC; -ADD JAR ../data/files/TestSerDe.jar; +ADD 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 INPUT16_CC(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties ('testserde.default.serialization.format'='\003', 'dummy.prop.not.used'='dummyy.val') STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1_cc.txt' INTO TABLE INPUT16_CC; +LOAD DATA LOCAL INPATH '../../data/files/kv1_cc.txt' INTO TABLE INPUT16_CC; SELECT INPUT16_CC.VALUE, INPUT16_CC.KEY FROM INPUT16_CC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q index fec44e97669dd..3dc7fec9f6669 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q @@ -1,5 +1,5 @@ create table apachelog(ipaddress STRING,identd STRING,user_name STRING,finishtime STRING,requestline string,returncode INT,size INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe' WITH SERDEPROPERTIES ( 'serialization.format'= 'org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol', 'quote.delim'= '("|\\[|\\])', 'field.delim'=' ', 'serialization.null.format'='-' ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/apache.access.log' INTO TABLE apachelog; +LOAD DATA LOCAL INPATH '../../data/files/apache.access.log' INTO TABLE apachelog; SELECT a.* FROM apachelog a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q index 0566ab18c2537..ff430abb8e8aa 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q @@ -1,6 +1,6 @@ CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; -ADD FILE ../data/scripts/input20_script; +ADD FILE ../../data/scripts/input20_script.py; EXPLAIN FROM ( @@ -12,7 +12,7 @@ FROM ( ) tmap INSERT OVERWRITE TABLE dest1 REDUCE tmap.key, tmap.value -USING 'input20_script' +USING 'python input20_script.py' AS key, value; FROM ( @@ -24,7 +24,7 @@ FROM ( ) tmap INSERT OVERWRITE TABLE dest1 REDUCE tmap.key, tmap.value -USING 'input20_script' +USING 'python input20_script.py' AS key, value; SELECT * FROM dest1 SORT BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q index d7c814e58061c..43cd01e684b37 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q @@ -1,7 +1,7 @@ CREATE TABLE src_null(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/null.txt' INTO TABLE src_null; +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE src_null; EXPLAIN SELECT * FROM src_null DISTRIBUTE BY c SORT BY d; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q index 853947be57a1f..8803e4dbeb8cf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q @@ -1,5 +1,5 @@ CREATE TABLE INPUT4(KEY STRING, VALUE STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4; EXPLAIN SELECT a.KEY2 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q index 7ab17515af5c7..8b6b21502001e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q @@ -1,6 +1,6 @@ CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; -ADD FILE ../data/scripts/input20_script; +ADD FILE ../../data/scripts/input20_script.py; EXPLAIN FROM ( @@ -12,7 +12,7 @@ FROM ( ) tmap INSERT OVERWRITE TABLE dest1 REDUCE tmap.key, tmap.value -USING 'input20_script' +USING 'python input20_script.py' AS (key STRING, value STRING); FROM ( @@ -24,7 +24,7 @@ FROM ( ) tmap INSERT OVERWRITE TABLE dest1 REDUCE tmap.key, tmap.value -USING 'input20_script' +USING 'python input20_script.py' AS (key STRING, value STRING); SELECT * FROM dest1 SORT BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q index 6fd136afec444..6ded61aa23990 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q @@ -1,6 +1,6 @@ create table documents(contents string) stored as textfile; -LOAD DATA LOCAL INPATH '../data/files/docurl.txt' INTO TABLE documents; +LOAD DATA LOCAL INPATH '../../data/files/docurl.txt' INTO TABLE documents; select url, count(1) @@ -8,7 +8,7 @@ FROM ( FROM documents MAP documents.contents - USING 'java -cp ../build/ql/test/classes org.apache.hadoop.hive.scripts.extracturl' AS (url, count) + USING 'java -cp ../util/target/classes/ org.apache.hadoop.hive.scripts.extracturl' AS (url, count) ) subq group by url; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q index 3584820acaf4a..f983aca847d95 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q @@ -1,7 +1,7 @@ CREATE TABLE T1(key STRING, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, value STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q index 08d6d97603045..1186bbbbe6bbd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q @@ -1,7 +1,7 @@ CREATE TABLE INPUT4(KEY STRING, VALUE STRING) STORED AS TEXTFILE; EXPLAIN -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4; EXPLAIN FORMATTED SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias; SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q index 4166cb5f94939..ab187b5d7e7d1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q @@ -2,15 +2,15 @@ create table tmp_insert_test (key string, value string) stored as textfile; -load data local inpath '../data/files/kv1.txt' into table tmp_insert_test; +load data local inpath '../../data/files/kv1.txt' into table tmp_insert_test; select * from tmp_insert_test; create table tmp_insert_test_p (key string, value string) partitioned by (ds string) stored as textfile; -load data local inpath '../data/files/kv1.txt' into table tmp_insert_test_p partition (ds = '2009-08-01'); +load data local inpath '../../data/files/kv1.txt' into table tmp_insert_test_p partition (ds = '2009-08-01'); select * from tmp_insert_test_p where ds= '2009-08-01' order by key, value; -load data local inpath '../data/files/kv2.txt' into table tmp_insert_test_p partition (ds = '2009-08-01'); +load data local inpath '../../data/files/kv2.txt' into table tmp_insert_test_p partition (ds = '2009-08-01'); select * from tmp_insert_test_p where ds= '2009-08-01' order by key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q index 5512dc328065d..3182bbef39812 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q @@ -1,7 +1,7 @@ drop table tst_src1; create table tst_src1 like src1; -load data local inpath '../data/files/kv1.txt' into table tst_src1 ; +load data local inpath '../../data/files/kv1.txt' into table tst_src1 ; select count(1) from tst_src1; -load data local inpath '../data/files/kv1.txt' into table tst_src1 ; +load data local inpath '../../data/files/kv1.txt' into table tst_src1 ; select count(1) from tst_src1; drop table tst_src1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q index 4557edc178b7f..2e975e58c1e93 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q @@ -3,4 +3,4 @@ CREATE TABLE dest(key INT, value STRING) STORED AS TEXTFILE; SET hive.output.file.extension=.txt; INSERT OVERWRITE TABLE dest SELECT src.* FROM src; -dfs -cat ../build/ql/test/data/warehouse/dest/*.txt \ No newline at end of file +dfs -cat ${system:test.warehouse.dir}/dest/*.txt diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q index 633a8c6edb5cc..334da264d6e55 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q @@ -2,8 +2,8 @@ SET hive.insert.into.multilevel.dirs=true; SET hive.output.file.extension=.txt; -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/x/y/z/' SELECT src.* FROM src; +INSERT OVERWRITE DIRECTORY 'target/data/x/y/z/' SELECT src.* FROM src; -dfs -cat ../build/ql/test/data/x/y/z/*.txt; +dfs -cat ${system:build.dir}/data/x/y/z/*.txt; -dfs -rmr ../build/ql/test/data/x; \ No newline at end of file +dfs -rmr ${system:build.dir}/data/x; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q index 8c57dd3f25aa6..b18d60aa74e73 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q @@ -1,4 +1,4 @@ CREATE TABLE INPUT4_CB(KEY STRING, VALUE STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' LINES TERMINATED BY '\012' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1_cb.txt' INTO TABLE INPUT4_CB; +LOAD DATA LOCAL INPATH '../../data/files/kv1_cb.txt' INTO TABLE INPUT4_CB; SELECT INPUT4_CB.VALUE, INPUT4_CB.KEY FROM INPUT4_CB; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q index 4f5824df5c9d5..b108cbd6b23e5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q @@ -1,2 +1,2 @@ -dfs -cat ../data/files/kv1.txt; +dfs -cat ../../data/files/kv1.txt; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q index 9a7ca5da1277f..87c55a26d7b83 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q @@ -1,7 +1,7 @@ -- test for internationalization -- kv4.txt contains the utf-8 character 0xE982B5E993AE which we are verifying later on CREATE TABLE INPUTDDL5(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE INPUTDDL5; +LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE INPUTDDL5; DESCRIBE INPUTDDL5; SELECT INPUTDDL5.name from INPUTDDL5; SELECT count(1) FROM INPUTDDL5 WHERE INPUTDDL5.name = _UTF-8 0xE982B5E993AE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q index d33ab8d9a78f4..6c709399a3c0d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q @@ -2,8 +2,8 @@ -- test for describe extended table partition -- test for alter table drop partition CREATE TABLE INPUTDDL6(KEY STRING, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-08'); DESCRIBE EXTENDED INPUTDDL6; DESCRIBE EXTENDED INPUTDDL6 PARTITION (ds='2008-04-08'); SHOW PARTITIONS INPUTDDL6; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q index 8a73935feec51..27e587a283cd3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q @@ -3,22 +3,22 @@ CREATE TABLE T1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1; SELECT COUNT(1) FROM T1; CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T2; SELECT COUNT(1) FROM T2; CREATE TABLE T3(name STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T3 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T3 PARTITION (ds='2008-04-09'); SELECT COUNT(1) FROM T3 where T3.ds='2008-04-09'; CREATE TABLE T4(name STRING) PARTITIONED BY(ds STRING) STORED AS SEQUENCEFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T4 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T4 PARTITION (ds='2008-04-09'); SELECT COUNT(1) FROM T4 where T4.ds='2008-04-09'; DESCRIBE EXTENDED T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q index 6ad70b5673f30..6b00f977c4c68 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q @@ -1,8 +1,8 @@ CREATE TABLE sourceTable (one string,two string) PARTITIONED BY (ds string,hr string); -load data local inpath '../data/files/kv1.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='11'); +load data local inpath '../../data/files/kv1.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='11'); -load data local inpath '../data/files/kv3.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='12'); +load data local inpath '../../data/files/kv3.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='12'); CREATE TABLE destinTable (one string,two string) PARTITIONED BY (ds string,hr string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q index 598d30eaebba2..bd1eb752879e6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q @@ -4,9 +4,9 @@ CREATE DATABASE db2; CREATE TABLE db1.sourceTable (one string,two string) PARTITIONED BY (ds string); -load data local inpath '../data/files/kv1.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); +load data local inpath '../../data/files/kv1.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); -load data local inpath '../data/files/kv3.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); +load data local inpath '../../data/files/kv3.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); CREATE TABLE db2.destinTable (one string,two string) PARTITIONED BY (ds string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q index e58b212e190bb..4ff0edcd61bca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q @@ -4,10 +4,10 @@ DROP TABLE insert_into3b; CREATE TABLE insert_into3a (key int, value string); CREATE TABLE insert_into3b (key int, value string); -EXPLAIN FROM src INSERT INTO TABLE insert_into3a SELECT * LIMIT 50 - INSERT INTO TABLE insert_into3b SELECT * LIMIT 100; -FROM src INSERT INTO TABLE insert_into3a SELECT * LIMIT 50 - INSERT INTO TABLE insert_into3b SELECT * LIMIT 100; +EXPLAIN FROM src INSERT INTO TABLE insert_into3a SELECT * ORDER BY key, value LIMIT 50 + INSERT INTO TABLE insert_into3b SELECT * ORDER BY key, value LIMIT 100; +FROM src INSERT INTO TABLE insert_into3a SELECT * ORDER BY key, value LIMIT 50 + INSERT INTO TABLE insert_into3b SELECT * ORDER BY key, value LIMIT 100; SELECT SUM(HASH(c)) FROM ( SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into3a ) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q index 25c127f67f980..6d069f5411d45 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q @@ -1,40 +1,40 @@ -insert overwrite local directory '../data/files/local_src_table_1' +insert overwrite local directory '../../data/files/local_src_table_1' select * from src ; -dfs -cat ../data/files/local_src_table_1/000000_0; +dfs -cat ../../data/files/local_src_table_1/000000_0; -insert overwrite local directory '../data/files/local_src_table_2' +insert overwrite local directory '../../data/files/local_src_table_2' row format delimited FIELDS TERMINATED BY ':' select * from src ; -dfs -cat ../data/files/local_src_table_2/000000_0; +dfs -cat ../../data/files/local_src_table_2/000000_0; create table array_table (a array, b array) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY ','; -load data local inpath "../data/files/array_table.txt" overwrite into table array_table; +load data local inpath "../../data/files/array_table.txt" overwrite into table array_table; -insert overwrite local directory '../data/files/local_array_table_1' +insert overwrite local directory '../../data/files/local_array_table_1' select * from array_table; -dfs -cat ../data/files/local_array_table_1/000000_0; +dfs -cat ../../data/files/local_array_table_1/000000_0; -insert overwrite local directory '../data/files/local_array_table_2' +insert overwrite local directory '../../data/files/local_array_table_2' ROW FORMAT DELIMITED FIELDS TERMINATED BY ':' COLLECTION ITEMS TERMINATED BY '#' select * from array_table; -dfs -cat ../data/files/local_array_table_2/000000_0; +dfs -cat ../../data/files/local_array_table_2/000000_0; -insert overwrite local directory '../data/files/local_array_table_2_withfields' +insert overwrite local directory '../../data/files/local_array_table_2_withfields' ROW FORMAT DELIMITED FIELDS TERMINATED BY ':' COLLECTION ITEMS TERMINATED BY '#' select b,a from array_table; -dfs -cat ../data/files/local_array_table_2_withfields/000000_0; +dfs -cat ../../data/files/local_array_table_2_withfields/000000_0; create table map_table (foo STRING , bar MAP) @@ -44,63 +44,63 @@ COLLECTION ITEMS TERMINATED BY ',' MAP KEYS TERMINATED BY ':' STORED AS TEXTFILE; -load data local inpath "../data/files/map_table.txt" overwrite into table map_table; +load data local inpath "../../data/files/map_table.txt" overwrite into table map_table; -insert overwrite local directory '../data/files/local_map_table_1' +insert overwrite local directory '../../data/files/local_map_table_1' select * from map_table; -dfs -cat ../data/files/local_map_table_1/000000_0; +dfs -cat ../../data/files/local_map_table_1/000000_0; -insert overwrite local directory '../data/files/local_map_table_2' +insert overwrite local directory '../../data/files/local_map_table_2' ROW FORMAT DELIMITED FIELDS TERMINATED BY ':' COLLECTION ITEMS TERMINATED BY '#' MAP KEYS TERMINATED BY '=' select * from map_table; -dfs -cat ../data/files/local_map_table_2/000000_0; +dfs -cat ../../data/files/local_map_table_2/000000_0; -insert overwrite local directory '../data/files/local_map_table_2_withfields' +insert overwrite local directory '../../data/files/local_map_table_2_withfields' ROW FORMAT DELIMITED FIELDS TERMINATED BY ':' COLLECTION ITEMS TERMINATED BY '#' MAP KEYS TERMINATED BY '=' select bar,foo from map_table; -dfs -cat ../data/files/local_map_table_2_withfields/000000_0; +dfs -cat ../../data/files/local_map_table_2_withfields/000000_0; -insert overwrite local directory '../data/files/local_array_table_3' +insert overwrite local directory '../../data/files/local_array_table_3' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.DelimitedJSONSerDe' STORED AS TEXTFILE select * from array_table; -dfs -cat ../data/files/local_array_table_3/000000_0; +dfs -cat ../../data/files/local_array_table_3/000000_0; -insert overwrite local directory '../data/files/local_map_table_3' +insert overwrite local directory '../../data/files/local_map_table_3' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.DelimitedJSONSerDe' STORED AS TEXTFILE select * from map_table; -dfs -cat ../data/files/local_map_table_3/000000_0; +dfs -cat ../../data/files/local_map_table_3/000000_0; -insert overwrite local directory '../data/files/local_rctable' +insert overwrite local directory '../../data/files/local_rctable' STORED AS RCFILE select value,key from src; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/local_rctable/temp; dfs -rmr ${system:test.tmp.dir}/local_rctable; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/local_rctable; -dfs -put ../data/files/local_rctable/000000_0 ${system:test.tmp.dir}/local_rctable/000000_0; +dfs -put ../../data/files/local_rctable/000000_0 ${system:test.tmp.dir}/local_rctable/000000_0; create external table local_rctable(value string, key string) STORED AS RCFILE LOCATION '${system:test.tmp.dir}/local_rctable'; -insert overwrite local directory '../data/files/local_rctable_out' +insert overwrite local directory '../../data/files/local_rctable_out' ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' select key,value from local_rctable; -dfs -cat ../data/files/local_rctable_out/000000_0; +dfs -cat ../../data/files/local_rctable_out/000000_0; drop table local_rctable; drop table array_table; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q index b403814adb4c9..4d1ae2186e796 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q @@ -1,9 +1,9 @@ CREATE TABLE join_1to1_1(key1 int, key2 int, value int); -LOAD DATA LOCAL INPATH '../data/files/in5.txt' INTO TABLE join_1to1_1; +LOAD DATA LOCAL INPATH '../../data/files/in5.txt' INTO TABLE join_1to1_1; CREATE TABLE join_1to1_2(key1 int, key2 int, value int); -LOAD DATA LOCAL INPATH '../data/files/in6.txt' INTO TABLE join_1to1_2; +LOAD DATA LOCAL INPATH '../../data/files/in6.txt' INTO TABLE join_1to1_2; set hive.outerjoin.supports.filters=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q new file mode 100644 index 0000000000000..0b0c53803d01e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q @@ -0,0 +1,41 @@ +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; + +explain select p1.p_name, p2.p_name +from part p1 , part p2; + +explain select p1.p_name, p2.p_name, p3.p_name +from part p1 ,part p2 ,part p3 +where p1.p_name = p2.p_name and p2.p_name = p3.p_name; + +explain select p1.p_name, p2.p_name, p3.p_name +from part p1 , (select p_name from part) p2 ,part p3 +where p1.p_name = p2.p_name and p2.p_name = p3.p_name; + +explain select p1.p_name, p2.p_name, p3.p_name +from part p1 , part p2 , part p3 +where p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name; + +explain select p1.p_name, p2.p_name, p3.p_name, p4.p_name +from part p1 , part p2 join part p3 on p2.p_name = p1.p_name join part p4 +where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2.p_partkey; + +explain select p1.p_name, p2.p_name, p3.p_name, p4.p_name +from part p1 join part p2 on p2.p_name = p1.p_name , part p3 , part p4 +where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2.p_partkey; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q index e4d95a51fa927..81e984e7eb018 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q @@ -1,8 +1,8 @@ create table tinyA(a bigint, b bigint) stored as textfile; create table tinyB(a bigint, bList array) stored as textfile; -load data local inpath '../data/files/tiny_a.txt' into table tinyA; -load data local inpath '../data/files/tiny_b.txt' into table tinyB; +load data local inpath '../../data/files/tiny_a.txt' into table tinyA; +load data local inpath '../../data/files/tiny_b.txt' into table tinyB; select * from tinyA; select * from tinyB; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q index a9b69c5b1c448..0c0962ceceebc 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q @@ -1,8 +1,8 @@ CREATE TABLE joinone(key1 int, key2 int, value int); -LOAD DATA LOCAL INPATH '../data/files/in5.txt' INTO TABLE joinone; +LOAD DATA LOCAL INPATH '../../data/files/in5.txt' INTO TABLE joinone; CREATE TABLE joinTwo(key1 int, key2 int, value int); -LOAD DATA LOCAL INPATH '../data/files/in6.txt' INTO TABLE joinTwo; +LOAD DATA LOCAL INPATH '../../data/files/in6.txt' INTO TABLE joinTwo; SELECT * FROM joinone JOIN joinTwo ON(joinone.key2=joinTwo.key2) ORDER BY joinone.key1 ASC, joinone.key2 ASC, joinone.value ASC, joinTwo.key1 ASC, joinTwo.key2 ASC, joinTwo.value ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q new file mode 100644 index 0000000000000..7f493671b80c1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q @@ -0,0 +1,30 @@ +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; + + + +explain select * +from part p1 join part p2 join part p3 on p1.p_name = p2.p_name and p2.p_name = p3.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_name = p1.p_name and p3.p_name = p2.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_partkey = 1 and p3.p_name = p2.p_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q new file mode 100644 index 0000000000000..ca280104d9a75 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q @@ -0,0 +1,24 @@ +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; + + +explain select * +from part p1 join part p2 join part p3 on p1.p_name = p2.p_name join part p4 on p2.p_name = p3.p_name and p1.p_name = p4.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_name = p1.p_name join part p4 on p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2.p_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q new file mode 100644 index 0000000000000..b308838d6243b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q @@ -0,0 +1,34 @@ +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; + + + +explain select * +from part p1 join part p2 join part p3 +where p1.p_name = p2.p_name and p2.p_name = p3.p_name; + +explain select * +from part p1 join part p2 join part p3 +where p2.p_name = p1.p_name and p3.p_name = p2.p_name; + +explain select * +from part p1 join part p2 join part p3 +where p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name; + +explain select * +from part p1 join part p2 join part p3 +where p2.p_partkey = 1 and p3.p_name = p2.p_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q new file mode 100644 index 0000000000000..477682e4e8ac7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q @@ -0,0 +1,26 @@ +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; + + +explain select * +from part p1 join part p2 join part p3 on p1.p_name = p2.p_name join part p4 +where p2.p_name = p3.p_name and p1.p_name = p4.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_name = p1.p_name join part p4 +where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2.p_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q new file mode 100644 index 0000000000000..1013f51a0d806 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q @@ -0,0 +1,52 @@ +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; + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +create table part3( + p3_partkey INT, + p3_name STRING, + p3_mfgr STRING, + p3_brand STRING, + p3_type STRING, + p3_size INT, + p3_container STRING, + p3_retailprice DOUBLE, + p3_comment STRING +); + +explain select * +from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name and p2_name = p3_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name and p3_name = p2_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_partkey + p_partkey = p1.p_partkey and p3_name = p2_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_partkey = 1 and p3_name = p2_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q new file mode 100644 index 0000000000000..6232357752851 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q @@ -0,0 +1,47 @@ +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; + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +create table part3( + p3_partkey INT, + p3_name STRING, + p3_mfgr STRING, + p3_brand STRING, + p3_type STRING, + p3_size INT, + p3_container STRING, + p3_retailprice DOUBLE, + p3_comment STRING +); + +explain select * +from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name join part p4 on p2_name = p3_name and p1.p_name = p4.p_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name join part p4 on p2_name = p3_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q new file mode 100644 index 0000000000000..6ac86042c0452 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q @@ -0,0 +1,56 @@ +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; + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +create table part3( + p3_partkey INT, + p3_name STRING, + p3_mfgr STRING, + p3_brand STRING, + p3_type STRING, + p3_size INT, + p3_container STRING, + p3_retailprice DOUBLE, + p3_comment STRING +); + +explain select * +from part p1 join part2 p2 join part3 p3 +where p1.p_name = p2_name and p2_name = p3_name; + +explain select * +from part p1 join part2 p2 join part3 p3 +where p2_name = p1.p_name and p3_name = p2_name; + +explain select * +from part p1 join part2 p2 join part3 p3 +where p2_partkey + p1.p_partkey = p1.p_partkey and p3_name = p2_name; + +explain select * +from part p1 join part2 p2 join part3 p3 +where p2_partkey = 1 and p3_name = p2_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q new file mode 100644 index 0000000000000..0db4d5e3cff1a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q @@ -0,0 +1,49 @@ +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; + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +create table part3( + p3_partkey INT, + p3_name STRING, + p3_mfgr STRING, + p3_brand STRING, + p3_type STRING, + p3_size INT, + p3_container STRING, + p3_retailprice DOUBLE, + p3_comment STRING +); + +explain select * +from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name join part p4 +where p2_name = p3_name and p1.p_name = p4.p_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name join part p4 +where p2_name = p3_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q index d54aa950a1ec2..49b6c6f920209 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q @@ -1,5 +1,5 @@ CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in3.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in3.txt' INTO TABLE myinput1; SELECT * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; @@ -55,10 +55,10 @@ SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.valu CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2; SET hive.optimize.bucketmapjoin = true; SET hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q index 31b0c8c91c10b..c4c239cae2d8a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q @@ -13,9 +13,9 @@ delimited fields terminated by ',' stored as textfile; create table hive_count (bar_id int, n int) row format delimited fields terminated by ',' stored as textfile; -load data local inpath '../data/files/hive_626_foo.txt' overwrite into table hive_foo; -load data local inpath '../data/files/hive_626_bar.txt' overwrite into table hive_bar; -load data local inpath '../data/files/hive_626_count.txt' overwrite into table hive_count; +load data local inpath '../../data/files/hive_626_foo.txt' overwrite into table hive_foo; +load data local inpath '../../data/files/hive_626_bar.txt' overwrite into table hive_bar; +load data local inpath '../../data/files/hive_626_count.txt' overwrite into table hive_count; explain select hive_foo.foo_name, hive_bar.bar_name, n from hive_foo join hive_bar on hive_foo.foo_id = diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q new file mode 100644 index 0000000000000..a0046dbc41332 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q @@ -0,0 +1,25 @@ + + +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 +); + +explain select p1.p_size, p2.p_size +from part p1 left outer join part p2 on p1.p_partkey = p2.p_partkey + right outer join part p3 on p2.p_partkey = p3.p_partkey and + p1.p_size > 10 +; + +explain select p1.p_size, p2.p_size +from part p1 left outer join part p2 on p1.p_partkey = p2.p_partkey + right outer join part p3 on p2.p_partkey = p3.p_partkey and + p1.p_size > 10 and p1.p_size > p2.p_size + 10 +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q index 4ff60713d6b44..047a769eb8e95 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q @@ -1,5 +1,5 @@ CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in1.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' INTO TABLE myinput1; SELECT * FROM myinput1 a JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; @@ -42,10 +42,10 @@ SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.valu CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2; SET hive.optimize.bucketmapJOIN = true; SET hive.optimize.bucketmapJOIN.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q index 05b57bce202d2..5e22517edbd72 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q @@ -1,7 +1,7 @@ set hive.nullsafe.equijoin=true; CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in8.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in8.txt' INTO TABLE myinput1; -- merging explain select * from myinput1 a join myinput1 b on a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value; @@ -31,10 +31,10 @@ SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key<=>b.value OR -- smbs CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/in8.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in9.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in8.txt' into table smb_input2; -LOAD DATA LOCAL INPATH '../data/files/in9.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in8.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in9.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in8.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in9.txt' into table smb_input2; SET hive.optimize.bucketmapJOIN = true; SET hive.optimize.bucketmapJOIN.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q index b92a79ba07ab2..b209c50b66194 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q @@ -6,9 +6,9 @@ CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; EXPLAIN FROM T1 a JOIN src c ON c.key+1=a.key SELECT a.key, a.val, c.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q index 238c0adad3128..ca1e65ebef6f0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q @@ -8,10 +8,10 @@ CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4; EXPLAIN SELECT /*+ STREAMTABLE(a) */ * diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q index 1bda28fbc3d5f..994be164aa62c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q @@ -8,10 +8,10 @@ CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4; EXPLAIN SELECT /*+ STREAMTABLE(a,c) */ * diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q index 126f356ef785a..16ef2046be35a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q @@ -2,9 +2,9 @@ CREATE TABLE T1(key1 STRING, val1 STRING) STORED AS TEXTFILE; CREATE TABLE T2(key2 STRING, val2 STRING) STORED AS TEXTFILE; CREATE TABLE T3(key3 STRING, val3 STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; set hive.auto.convert.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q index 8314161975761..c95a13b9cac5a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q @@ -7,14 +7,14 @@ create table dim5(f9 int, f10 int); create table dim6(f11 int, f12 int); create table dim7(f13 int, f14 int); -LOAD DATA LOCAL INPATH '../data/files/fact-data.txt' INTO TABLE fact; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim1; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim2; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim3; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim4; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim5; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim6; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim7; +LOAD DATA LOCAL INPATH '../../data/files/fact-data.txt' INTO TABLE fact; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim1; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim2; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim3; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim4; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim5; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim6; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim7; set hive.auto.convert.join=true; set hive.auto.convert.join.noconditionaltask=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q index 11e54a5b04c07..df7343259ae75 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + --HIVE-2608 Do not require AS a,b,c part in LATERAL VIEW EXPLAIN SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2; SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q index 7be86a6f10ea2..65ae518cd5be2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q @@ -11,3 +11,7 @@ SELECT value, myCol FROM (SELECT * FROM srcpart LATERAL VIEW explode(array(1,2,3 EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE key='0'; SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE key='0'; + +-- HIVE-4293 Predicates following UDTF operator are removed by PPD +EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0'; +SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q new file mode 100644 index 0000000000000..7f31797f314b3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q @@ -0,0 +1,19 @@ +set hive.mapred.supports.subdirectories=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; +set hive.stats.dbclass=fs; +-- Tests truncating a column from a list bucketing table + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; + +ALTER TABLE test_tab SKEWED BY (key) ON ("484") STORED AS DIRECTORIES; + +INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src; + +describe formatted test_tab partition (part='1'); + +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q index f49766771067b..5623cbfac51a7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; --1. testLagWithPTFWindowing select p_mfgr, p_name, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q index 6ef3bdb107a12..e53abce763865 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q @@ -11,7 +11,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. testLeadUDAF select p_mfgr, p_retailprice, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q index abe3d3317157c..0c16fb8dfd988 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q @@ -9,9 +9,9 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; CREATE TABLE things (id INT, name STRING) partitioned by (ds string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; -load data local inpath '../data/files/sales.txt' INTO TABLE sales; -load data local inpath '../data/files/things.txt' INTO TABLE things partition(ds='2011-10-23'); -load data local inpath '../data/files/things2.txt' INTO TABLE things partition(ds='2011-10-24'); +load data local inpath '../../data/files/sales.txt' INTO TABLE sales; +load data local inpath '../../data/files/things.txt' INTO TABLE things partition(ds='2011-10-23'); +load data local inpath '../../data/files/things2.txt' INTO TABLE things partition(ds='2011-10-24'); SELECT name,id FROM sales ORDER BY name ASC, id ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q index 5813ca3c619cc..c9ebe0e8fad12 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q @@ -1,7 +1,7 @@ CREATE TABLE T1(key INT); -LOAD DATA LOCAL INPATH '../data/files/leftsemijoin_mr_t1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t1.txt' INTO TABLE T1; CREATE TABLE T2(key INT); -LOAD DATA LOCAL INPATH '../data/files/leftsemijoin_mr_t2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t2.txt' INTO TABLE T2; -- Run this query using TestMinimrCliDriver diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q new file mode 100644 index 0000000000000..e91adab59ddfa --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q @@ -0,0 +1,7 @@ +set hive.limit.query.max.table.partition=1; + +explain select ds from srcpart where hr=11 and ds='2008-04-08'; +select ds from srcpart where hr=11 and ds='2008-04-08'; + +explain select distinct hr from srcpart; +select distinct hr from srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q index e4d0aa06bde87..adfe1e63e889f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q @@ -22,12 +22,17 @@ select value,avg(key + 1) from src group by value order by value limit 20; -- distincts explain -select distinct(key) from src limit 20; -select distinct(key) from src limit 20; +select distinct(cdouble) from alltypesorc limit 20; +select distinct(cdouble) from alltypesorc limit 20; explain -select key, count(distinct(key)) from src group by key limit 20; -select key, count(distinct(key)) from src group by key limit 20; +select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20; +select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20; + +-- multi distinct +explain +select ctinyint, count(distinct(cstring1)), count(distinct(cstring2)) from alltypesorc group by ctinyint limit 20; +select ctinyint, count(distinct(cstring1)), count(distinct(cstring2)) from alltypesorc group by ctinyint limit 20; -- limit zero explain diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q index a86ddf14047ef..e17ded1ee1b9b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q @@ -16,7 +16,3 @@ CREATE TABLE dest_3(key STRING, c1 INT); EXPLAIN FROM src INSERT OVERWRITE TABLE dest_2 SELECT value, sum(key) GROUP BY value INSERT OVERWRITE TABLE dest_3 SELECT value, sum(key) GROUP BY value limit 20; - --- nagative, multi distinct -explain -select count(distinct key)+count(distinct value) from src limit 20; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q index 3a39f42b2a77d..80aba5d4a526f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q @@ -48,13 +48,13 @@ set hive.optimize.listbucketing=true; explain extended select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; -select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value; +select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; -- 51 and val_51 in the table so skewed data for 51 and val_14 should be none -- but query should succeed for 51 or 51 and val_14 -select * from srcpart where ds = '2008-04-08' and key = '51' ORDER BY key, value; +select * from srcpart where ds = '2008-04-08' and key = '51' ORDER BY key, value, ds, hr; select * from list_bucketing_static_part where key = '51' ORDER BY key, value, ds, hr; -select * from srcpart where ds = '2008-04-08' and key = '51' and value = 'val_14' ORDER BY key, value; +select * from srcpart where ds = '2008-04-08' and key = '51' and value = 'val_14' ORDER BY key, value, ds, hr; select * from list_bucketing_static_part where key = '51' and value = 'val_14' ORDER BY key, value, ds, hr; -- queries with < <= > >= should work for skewed test although we don't benefit from pruning diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q index 918c817e49eff..380d148ac9f43 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q @@ -65,7 +65,7 @@ set hive.optimize.listbucketing=true; explain extended select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; -select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value; +select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; -- clean up drop table list_bucketing_static_part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q index a6ad4b8485a26..08b21dc689424 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT -1BD, 0BD, 1BD, 3.14BD, -3.14BD, 99999999999999999BD, 99999999999999999.9999999999999BD, 1E-99BD, 1E99BD FROM src LIMIT 1; SELECT -1BD, 0BD, 1BD, 3.14BD, -3.14BD, 99999999999999999BD, 99999999999999999.9999999999999BD, 1E-99BD, 1E99BD FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q index 08836127b9958..766da699ea250 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 FROM src LIMIT 1; SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q index 9da622e246793..5fd0cfabf3076 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT 100, 100Y, 100S, 100L FROM src LIMIT 1; SELECT 100, 100Y, 100S, 100L FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q index 21f0890ada168..c57dc572529e1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT 'face''book', 'face' 'book', 'face' 'book', "face""book", "face" "book", "face" diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q index 7da363183ccf6..653918afc0fa3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q @@ -4,7 +4,7 @@ FIELDS TERMINATED BY '9' STORED AS TEXTFILE; -- this query loads native binary data, stores in a table and then queries it. Note that string.txt contains binary data. Also uses transform clause and then length udf. -LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE mytable; +LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE mytable; create table dest1 (key binary, value int); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q index 52b4937d4a9dd..5f0a015693d9f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q @@ -23,8 +23,8 @@ insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, v show partitions nzhang_part1; show partitions nzhang_part2; -select * from nzhang_part1 where ds is not null and hr is not null; -select * from nzhang_part2 where ds is not null and hr is not null; +select * from nzhang_part1 where ds is not null and hr is not null order by ds, hr, key; +select * from nzhang_part2 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q index 9517664675d69..dd84599e69e6d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q @@ -19,6 +19,6 @@ insert overwrite table nzhang_part10 partition(ds='2008-12-31', hr) select key, show partitions nzhang_part10; -select * from nzhang_part10 where ds is not null and hr is not null; +select * from nzhang_part10 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q index e4c8c17f63048..29f951aa69f67 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q @@ -14,6 +14,6 @@ insert overwrite table nzhang_part3 partition (ds, hr) select key, value, ds, hr insert overwrite table nzhang_part3 partition (ds, hr) select key, value, ds, hr from srcpart where ds is not null and hr is not null; -select * from nzhang_part3 where ds is not null and hr is not null; +select * from nzhang_part3 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q index 3f3a0c8d51b41..942c245db8b64 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q @@ -17,8 +17,8 @@ insert overwrite table nzhang_part4 partition (ds, hr) select key, value, ds, hr insert overwrite table nzhang_part4 partition (ds, hr) select key, value, ds, hr from srcpart where ds is not null and hr is not null; show partitions nzhang_part4; -select * from nzhang_part4 where ds='2008-04-08' and hr is not null; +select * from nzhang_part4 where ds='2008-04-08' and hr is not null order by ds, hr, key; -select * from nzhang_part4 where ds is not null and hr is not null; +select * from nzhang_part4 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q index 8073500c0bf07..6768e4373a056 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q @@ -20,5 +20,5 @@ insert overwrite table nzhang_part8 partition(ds='2008-12-31', hr) select key, v show partitions nzhang_part8; -select * from nzhang_part8 where ds is not null and hr is not null; +select * from nzhang_part8 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q index 01fa596cdf04a..4680033cbd541 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q @@ -19,5 +19,5 @@ insert overwrite table nzhang_part9 partition (ds, hr) select key, value, ds, hr show partitions nzhang_part9; -select * from nzhang_part9 where ds is not null and hr is not null; +select * from nzhang_part9 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q index 6d2a8b82d33c1..35eb2198081f0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q @@ -2,4 +2,4 @@ create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored alter table hive_test_src add partition (pcol1 = 'test_part'); set hive.security.authorization.enabled=true; grant Update on table hive_test_src to user hive_test_user; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); +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/clientpositive/load_file_with_space_in_the_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q index 3b8951a1a782f..6bac47fb9052a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q @@ -2,4 +2,5 @@ CREATE TABLE load_file_with_space_in_the_name(name STRING, age INT); -LOAD DATA LOCAL INPATH '../data/files/person age.txt' INTO TABLE load_file_with_space_in_the_name; +LOAD DATA LOCAL INPATH '../../data/files/person age.txt' INTO TABLE load_file_with_space_in_the_name; +LOAD DATA LOCAL INPATH '../../data/files/person+age.txt' INTO TABLE load_file_with_space_in_the_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q index c1ac29c172f60..2f06ca464ff99 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q @@ -2,9 +2,9 @@ create table load_overwrite (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/load_overwrite'; create table load_overwrite2 (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/load2_overwrite2'; -load data local inpath '../data/files/kv1.txt' into table load_overwrite; -load data local inpath '../data/files/kv2.txt' into table load_overwrite; -load data local inpath '../data/files/kv3.txt' into table load_overwrite; +load data local inpath '../../data/files/kv1.txt' into table load_overwrite; +load data local inpath '../../data/files/kv2.txt' into table load_overwrite; +load data local inpath '../../data/files/kv3.txt' into table load_overwrite; show table extended like load_overwrite; desc extended load_overwrite; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q index 7255324d1653d..a75758a0728d5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q @@ -4,17 +4,17 @@ create table result (key string, value string); create table loader (key string, value string); -load data local inpath '../data/files/kv1.txt' into table loader; +load data local inpath '../../data/files/kv1.txt' into table loader; load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result; show table extended like result; -load data local inpath '../data/files/kv1.txt' into table loader; +load data local inpath '../../data/files/kv1.txt' into table loader; load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result; show table extended like result; -load data local inpath '../data/files/kv1.txt' into table loader; +load data local inpath '../../data/files/kv1.txt' into table loader; load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result; show table extended like result; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q new file mode 100644 index 0000000000000..51a803130a660 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q @@ -0,0 +1,20 @@ +--HIVE 6209 + +drop table target; +drop table temp; + +create table target (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/target'; +create table temp (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/temp'; + +set fs.pfile.impl.disable.cache=false; + +load data local inpath '../../data/files/kv1.txt' into table temp; +load data inpath '${system:test.tmp.dir}/temp/kv1.txt' overwrite into table target; +select count(*) from target; + +load data local inpath '../../data/files/kv2.txt' into table temp; +load data inpath '${system:test.tmp.dir}/temp/kv2.txt' overwrite into table target; +select count(*) from target; + +drop table target; +drop table temp; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q index cce297cca46db..55ac1a8185a46 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q @@ -1,9 +1,10 @@ -dfs -mkdir hdfs:///tmp/test/; +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/; -dfs -copyFromLocal ../data/files hdfs:///tmp/test/.; +dfs -copyFromLocal ../../data/files hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/.; CREATE TABLE load_file_with_space_in_the_name(name STRING, age INT); -LOAD DATA INPATH 'hdfs:///tmp/test/files/person age.txt' INTO TABLE load_file_with_space_in_the_name; +LOAD DATA INPATH 'hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/files/person age.txt' INTO TABLE load_file_with_space_in_the_name; +LOAD DATA INPATH 'hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/files/person+age.txt' INTO TABLE load_file_with_space_in_the_name; -dfs -rmr hdfs:///tmp/test; +dfs -rmr hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q index 40d8210e57d2a..fdee45114bb15 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q @@ -1,4 +1,4 @@ create table hive_test_src ( col1 string ) stored as textfile; set hive.security.authorization.enabled=true; grant Update on table hive_test_src to user hive_test_user; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +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/clientpositive/load_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q index 73853f15a94bb..080c78496a653 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q @@ -5,11 +5,11 @@ show table extended like load_overwrite; select count(*) from load_overwrite; -load data local inpath '../data/files/kv1.txt' into table load_overwrite; +load data local inpath '../../data/files/kv1.txt' into table load_overwrite; show table extended like load_overwrite; select count(*) from load_overwrite; -load data local inpath '../data/files/kv1.txt' overwrite into table load_overwrite; +load data local inpath '../../data/files/kv1.txt' overwrite into table load_overwrite; show table extended like load_overwrite; select count(*) from load_overwrite; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q index ff54324a5a50c..cee5873ca5b9f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q @@ -1,4 +1,4 @@ create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; set hive.security.authorization.enabled=true; grant Update on table hive_test_src to user hive_test_user; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); +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/clientpositive/loadpart1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q index 0813bb23c3746..735befef6f9c9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q @@ -2,7 +2,7 @@ create table hive_test_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; create table hive_test_dst ( col1 string ) partitioned by ( pcol1 string , pcol2 string) stored as sequencefile; insert overwrite table hive_test_dst partition ( pcol1='test_part', pCol2='test_Part') select col1 from hive_test_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q new file mode 100644 index 0000000000000..a252eaa00d77f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q @@ -0,0 +1,9 @@ + +create table hive_test ( col1 string ) partitioned by ( pcol1 string , pcol2 string) stored as textfile; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test partition (pcol1='part1',pcol2='part1') ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test partition (pcol2='part2',pcol1='part2') ; +select * from hive_test where pcol1='part1' and pcol2='part1'; +select * from hive_test where pcol1='part2' and pcol2='part2'; + + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q index 6e4df215479bb..cc9c1fec3bc42 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q @@ -1,6 +1,6 @@ set hive.cli.errors.ignore=true; -ADD FILE ../data/scripts/error_script; +ADD FILE ../../data/scripts/error_script; -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19, 0.20, 0.20S, 0.23) -- (this test is flaky so it is currently disabled for all Hadoop versions) @@ -14,7 +14,7 @@ FROM src; DESCRIBE loadpart1; SHOW PARTITIONS loadpart1; -LOAD DATA LOCAL INPATH '../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); +LOAD DATA LOCAL INPATH '../../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); SHOW PARTITIONS loadpart1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q index fd0f7f2b0cdd4..47b05ff4490fb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + CREATE TEMPORARY MACRO SIGMOID (x DOUBLE) 1.0 / (1.0 + EXP(-x)); SELECT SIGMOID(2) FROM src LIMIT 1; EXPLAIN SELECT SIGMOID(2) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q new file mode 100644 index 0000000000000..f56f074616678 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q @@ -0,0 +1,14 @@ + +set hive.auto.convert.join=true; +set hive.auto.convert.join.use.nonstaged=false; + +add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar; + +CREATE TABLE t1 (a string, b string) +ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' +; +LOAD DATA LOCAL INPATH "../../data/files/sample.json" INTO TABLE t1; +select * from src join t1 on src.key =t1.a; +drop table t1; +set hive.auto.convert.join=false; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q new file mode 100644 index 0000000000000..b65a7be2d25cd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q @@ -0,0 +1,35 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000000; + +CREATE TABLE over1k(t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal(4,2), + bin binary) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE over1k; + +CREATE TABLE t1(dec decimal(4,2)) STORED AS ORC; +INSERT INTO TABLE t1 select dec from over1k; +CREATE TABLE t2(dec decimal(4,0)) STORED AS ORC; +INSERT INTO TABLE t2 select dec from over1k; + +explain +select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec); + +set hive.mapjoin.optimized.keys=false; + +select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec); + +set hive.mapjoin.optimized.keys=true; + +select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q index d6811d493263f..a9e1960a5bb05 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q @@ -1,4 +1,5 @@ -set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.MapJoinCounterHook ; +set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.MapJoinCounterHook,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; + drop table dest1; CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q index 3f87db28ed2e2..1eb95f6378669 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q @@ -6,6 +6,14 @@ set hive.auto.convert.join.noconditionaltask.size=10000; explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key); -explain select count(*) from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) group by ds; +explain +select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450'; +select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450'; + +explain +select count(*) from srcpart join src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds; +select count(*) from srcpart join src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds; + +set hive.mapjoin.lazy.hashtable=false; select count(*) from srcpart join src src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q new file mode 100644 index 0000000000000..b23361724e669 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q @@ -0,0 +1,16 @@ + +set hive.auto.convert.join = true; + +create table src0 like src; +insert into table src0 select * from src where src.key < 10; + +set hive.mapjoin.check.memory.rows=1; + +explain +select src1.key as k1, src1.value as v1, src2.key, src2.value +from src0 src1 inner join src0 src2 on src1.key = src2.key order by k1, v1; + +select src1.key as k1, src1.value as v1, src2.key, src2.value +from src0 src1 inner join src0 src2 on src1.key = src2.key order by k1, v1; + +drop table src0; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q index 9980946057fe1..aed89905238b2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q @@ -11,9 +11,9 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; CREATE TABLE z (id INT, name STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; -load data local inpath '../data/files/x.txt' INTO TABLE x; -load data local inpath '../data/files/y.txt' INTO TABLE y; -load data local inpath '../data/files/z.txt' INTO TABLE z; +load data local inpath '../../data/files/x.txt' INTO TABLE x; +load data local inpath '../../data/files/y.txt' INTO TABLE y; +load data local inpath '../../data/files/z.txt' INTO TABLE z; set hive.auto.convert.join=true; set hive.auto.convert.join.noconditionaltask=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q index aacd0cd68fc50..c5c7ea202fa76 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q @@ -37,7 +37,7 @@ where ds is not null; show partitions merge_src_part2; -select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC; +select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC, ds ASC; drop table merge_src_part2; @@ -54,4 +54,4 @@ select key, value, ds; show partitions merge_src_part2; -select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC; +select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC, ds ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q index 744783bd62f8d..5a167aa0f08b3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q @@ -11,14 +11,14 @@ insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, v insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08'; -select * from nzhang_part; +select * from nzhang_part ORDER BY key, value, ds, hr; explain insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08'; insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08'; -select * from nzhang_part; +select * from nzhang_part ORDER BY key, value, ds, hr; explain insert overwrite table nzhang_part partition (ds='2010-08-15', hr) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q index 1379426b601f0..ae319865f6e7b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q @@ -5,15 +5,16 @@ create table srcpart_merge_dp like srcpart; create table merge_dynamic_part like srcpart; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set hive.merge.smallfiles.avgsize=1000000000; +set hive.optimize.sort.dynamic.partition=false; explain insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08'; insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q index b51c70ed03a46..73a71e6265d50 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q @@ -5,12 +5,12 @@ create table srcpart_merge_dp like srcpart; create table merge_dynamic_part like srcpart; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket0.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket0.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; @@ -18,6 +18,7 @@ set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=3000; set hive.exec.compress.output=false; +set hive.optimize.sort.dynamic.partition=false; explain insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q index b3bcf01ea043d..43be59e5d5f40 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q @@ -5,20 +5,20 @@ create table srcpart_merge_dp like srcpart; create table merge_dynamic_part like srcpart; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); - -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); - -load data local inpath '../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11); -load data local inpath '../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11); -load data local inpath '../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12); -load data local inpath '../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); + +load data local inpath '../../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11); +load data local inpath '../../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11); +load data local inpath '../../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12); +load data local inpath '../../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12); show partitions srcpart_merge_dp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q index ef769a042d7ce..589717096a9d9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q @@ -9,15 +9,15 @@ alter table srcpart_merge_dp_rc set fileformat RCFILE; create table merge_dynamic_part like srcpart; alter table merge_dynamic_part set fileformat RCFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); - -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); insert overwrite table srcpart_merge_dp_rc partition (ds = '2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds = '2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q index a196fa05288b7..9f64724563865 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q @@ -8,12 +8,12 @@ alter table srcpart_merge_dp_rc set fileformat RCFILE; create table merge_dynamic_part like srcpart; alter table merge_dynamic_part set fileformat RCFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); insert overwrite table srcpart_merge_dp_rc partition (ds = '2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds = '2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q index 41be152e7871d..e2da61a783950 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q @@ -1,8 +1,8 @@ create table tmp_meta_export_listener_drop_test (foo string); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/HIVE-3427; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/data/exports/HIVE-3427; set hive.metastore.pre.event.listeners=org.apache.hadoop.hive.ql.parse.MetaDataExportListener; -set hive.metadata.export.location=../build/ql/test/data/exports/HIVE-3427; +set hive.metadata.export.location=${system:test.tmp.dir}/data/exports/HIVE-3427; set hive.move.exported.metadata.to.trash=false; drop table tmp_meta_export_listener_drop_test; -dfs -rmr ../build/ql/test/data/exports/HIVE-3427; +dfs -rmr ${system:test.tmp.dir}/data/exports/HIVE-3427; set hive.metastore.pre.event.listeners=; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q new file mode 100644 index 0000000000000..b549a56232108 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q @@ -0,0 +1,77 @@ +set hive.stats.dbclass=fs; +set hive.compute.query.using.stats=true; +set hive.stats.autogather=true; +create table over10k( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) + row format delimited + fields terminated by '|'; + +load data local inpath '../../data/files/over10k' into table over10k; + +create table stats_tbl( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary); + +create table stats_tbl_part( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) partitioned by (dt string); + + +insert overwrite table stats_tbl select * from over10k; + +insert into table stats_tbl_part partition (dt='2010') select * from over10k where t>0 and t<30; +insert into table stats_tbl_part partition (dt='2011') select * from over10k where t>30 and t<60; +insert into table stats_tbl_part partition (dt='2012') select * from over10k where t>60; + +explain +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b) from stats_tbl; +explain +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b) from stats_tbl_part; + +analyze table stats_tbl compute statistics for columns t,si,i,b,f,d,bo,s,bin; +analyze table stats_tbl_part partition(dt='2010') compute statistics for columns t,si,i,b,f,d,bo,s,bin; +analyze table stats_tbl_part partition(dt='2011') compute statistics for columns t,si,i,b,f,d,bo,s,bin; +analyze table stats_tbl_part partition(dt='2012') compute statistics for columns t,si,i,b,f,d,bo,s,bin; + +explain +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl; +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl; +explain +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part; +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part; + +explain select count(ts) from stats_tbl_part; + +drop table stats_tbl; +drop table stats_tbl_part; + +set hive.compute.query.using.stats=false; +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q new file mode 100644 index 0000000000000..09f4bff616a50 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q @@ -0,0 +1,51 @@ +set hive.stats.dbclass=fs; +set hive.compute.query.using.stats=true; +create table over10k( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) + row format delimited + fields terminated by '|'; + +load data local inpath '../../data/files/over10k' into table over10k; + +create table stats_tbl_part( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) partitioned by (dt int); + + +from over10k +insert overwrite table stats_tbl_part partition (dt=2010) select t,si,i,b,f,d,bo,s,ts,dec,bin where t>0 and t<30 +insert overwrite table stats_tbl_part partition (dt=2014) select t,si,i,b,f,d,bo,s,ts,dec,bin where t > 30 and t<60; + +analyze table stats_tbl_part partition(dt) compute statistics; +analyze table stats_tbl_part partition(dt=2010) compute statistics for columns t,si,i,b,f,d,bo,s,bin; +analyze table stats_tbl_part partition(dt=2014) compute statistics for columns t,si,i,b,f,d,bo,s,bin; + +explain +select count(*), count(1), sum(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt = 2010; +select count(*), count(1), sum(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt = 2010; +explain +select count(*), count(1), sum(1), sum(2), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt > 2010; +select count(*), count(1), sum(1), sum(2), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt > 2010; + +drop table stats_tbl_part; +set hive.compute.query.using.stats=false; +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q index 067c143c0e684..2a6059b3a37ff 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q @@ -15,7 +15,7 @@ GROUP BY key, value, ds, hr; show partitions nzhang_t1; show partitions nzhang_t2; -select * from nzhang_t1; -select * from nzhang_t2; +select * from nzhang_t1 order by key, value; +select * from nzhang_t2 order by key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q new file mode 100644 index 0000000000000..9f068cc71394e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q @@ -0,0 +1,59 @@ +-- simple query with multiple reduce stages +EXPLAIN SELECT key, count(value) as cnt FROM src GROUP BY key ORDER BY cnt; +SELECT key, count(value) as cnt FROM src GROUP BY key ORDER BY cnt; + +set hive.auto.convert.join=false; +-- join query with multiple reduce stages; +EXPLAIN SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt; +SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt; + +set hive.auto.convert.join=true; +-- same query with broadcast join +EXPLAIN SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt; +SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt; + +set hive.auto.convert.join=false; +-- query with multiple branches in the task dag +EXPLAIN +SELECT * +FROM + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s1 + JOIN + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s2 + JOIN + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s3 + ON (s1.key = s2.key and s1.key = s3.key) +WHERE + s1.cnt > 1 +ORDER BY s1.key; + +SELECT * +FROM + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s1 + JOIN + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s2 + JOIN + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s3 + ON (s1.key = s2.key and s1.key = s3.key) +WHERE + s1.cnt > 1 +ORDER BY s1.key; + +set hive.auto.convert.join=true; +-- query with broadcast join in the reduce stage +EXPLAIN +SELECT * +FROM + (SELECT key, count(value) as cnt FROM src GROUP BY key) s1 + JOIN src ON (s1.key = src.key); + +SELECT * +FROM + (SELECT key, count(value) as cnt FROM src GROUP BY key) s1 + JOIN src ON (s1.key = src.key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q index 9a0a792a91897..455f550ae3ac5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q @@ -1,3 +1,5 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; + create table smallTbl1(key string, value string); insert overwrite table smallTbl1 select * from src where key < 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q index ce6cf6d8d6c0d..141db4db0a3f3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q @@ -1,3 +1,4 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; set hive.auto.convert.join=true; set hive.auto.convert.join.noconditionaltask=true; set hive.auto.convert.join.noconditionaltask.size=6000; @@ -187,3 +188,28 @@ FROM part_table x JOIN src1 y ON (x.key = y.key); SELECT count(*) FROM part_table x JOIN src1 y ON (x.key = y.key); +set hive.auto.convert.join.noconditionaltask.size=10000000; +set hive.optimize.correlation=false; +-- HIVE-5891 Alias conflict when merging multiple mapjoin tasks into their common +-- child mapred task +EXPLAIN +SELECT * FROM ( + SELECT c.key FROM + (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp + JOIN src c ON tmp.key=c.key + UNION ALL + SELECT c.key FROM + (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp + JOIN src c ON tmp.key=c.key +) x; + +SELECT * FROM ( + SELECT c.key FROM + (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp + JOIN src c ON tmp.key=c.key + UNION ALL + SELECT c.key FROM + (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp + JOIN src c ON tmp.key=c.key +) x; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q index b94fbb7b8be43..6fd76b859e4e4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q @@ -17,6 +17,6 @@ describe nestedcomplex; describe extended nestedcomplex; -load data local inpath '../data/files/nested_complex.txt' overwrite into table nestedcomplex; +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/clientpositive/newline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q index 722ecf6d972ca..11168fcd3b05c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q @@ -1,4 +1,4 @@ -add file ../data/scripts/newline.py; +add file ../../data/scripts/newline.py; set hive.transform.escape.input=true; create table tmp_tmp(key string, value string) stored as rcfile; @@ -10,10 +10,10 @@ select * from tmp_tmp ORDER BY key ASC, value ASC; drop table tmp_tmp; -add file ../data/scripts/escapednewline.py; -add file ../data/scripts/escapedtab.py; -add file ../data/scripts/doubleescapedtab.py; -add file ../data/scripts/escapedcarriagereturn.py; +add file ../../data/scripts/escapednewline.py; +add file ../../data/scripts/escapedtab.py; +add file ../../data/scripts/doubleescapedtab.py; +add file ../../data/scripts/escapedcarriagereturn.py; create table tmp_tmp(key string, value string) stored as rcfile; insert overwrite table tmp_tmp diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q new file mode 100644 index 0000000000000..e6343e2f535c0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q @@ -0,0 +1,9 @@ +set hive.fetch.task.conversion=more; + +explain select * from srcpart where ds='2008-04-08' AND hr='11' limit 10; +explain select cast(key as int) * 10, upper(value) from src limit 10; + +set hive.fetch.task.conversion.threshold=100; + +explain select * from srcpart where ds='2008-04-08' AND hr='11' limit 10; +explain select cast(key as int) * 10, upper(value) from src limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q index 9cb89da373d29..e33b4bfcab7ba 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q @@ -1,12 +1,12 @@ CREATE TABLE table(string string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/docurl.txt' INTO TABLE table; +LOAD DATA LOCAL INPATH '../../data/files/docurl.txt' INTO TABLE table; SELECT table, count(1) FROM ( FROM table SELECT TRANSFORM (table.string) - USING 'java -cp ../build/ql/test/classes org.apache.hadoop.hive.scripts.extracturl' AS (table, count) + USING 'java -cp ../util/target/classes/ org.apache.hadoop.hive.scripts.extracturl' AS (table, count) ) subq GROUP BY table; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q new file mode 100644 index 0000000000000..aa79674409d2d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q @@ -0,0 +1,4 @@ +CREATE TABLE dest1(c string, key INT, value DOUBLE) STORED AS TEXTFILE; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.key, sum(src.value) WHERE src.key < 100 group by key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q index 48c39b81fdd8f..bd0cb8d12d88f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q @@ -2,10 +2,10 @@ EXPLAIN SELECT ARRAY(NULL, 0), ARRAY(NULL, ARRAY()), ARRAY(NULL, MAP()), ARRAY(NULL, STRUCT(0)) - FROM src LIMIT 1; + FROM src tablesample (1 rows); SELECT ARRAY(NULL, 0), ARRAY(NULL, ARRAY()), ARRAY(NULL, MAP()), ARRAY(NULL, STRUCT(0)) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q index fa4a8639446e5..4b43d608e86df 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q @@ -3,7 +3,7 @@ create table temp_null(a int) stored as textfile; -load data local inpath '../data/files/test.dat' overwrite into table temp_null; +load data local inpath '../../data/files/test.dat' overwrite into table temp_null; select null, null from temp_null; @@ -15,8 +15,8 @@ create table tt_b(a int, b string) row format serde "org.apache.hadoop.hive.serd insert overwrite table tt_b select null, null from temp_null; select * from tt_b; -insert overwrite directory "../build/ql/test/data/warehouse/null_columns.out" select null, null from temp_null; -dfs -cat ../build/ql/test/data/warehouse/null_columns.out/*; +insert overwrite directory "target/warehouse/null_columns.out" select null, null from temp_null; +dfs -cat ${system:test.warehouse.dir}/null_columns.out/*; create table temp_null2 (key string, value string) partitioned by (ds string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q new file mode 100644 index 0000000000000..c9a7dab5eb9e7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q @@ -0,0 +1,24 @@ +-- base table with null data +DROP TABLE IF EXISTS base_tab; +CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab; +DESCRIBE EXTENDED base_tab; + +-- table with non-default null format +DROP TABLE IF EXISTS null_tab1; +EXPLAIN CREATE TABLE null_tab1(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull'; +CREATE TABLE null_tab1(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull'; +DESCRIBE EXTENDED null_tab1; +SHOW CREATE TABLE null_tab1; + +-- load null data from another table and verify that the null is stored in the expected format +INSERT OVERWRITE TABLE null_tab1 SELECT a,b FROM base_tab; +dfs -cat ${system:test.warehouse.dir}/null_tab1/*; +SELECT * FROM null_tab1; +-- alter the null format and verify that the old null format is no longer in effect +ALTER TABLE null_tab1 SET SERDEPROPERTIES ( 'serialization.null.format'='foo'); +SELECT * FROM null_tab1; + + +DROP TABLE null_tab1; +DROP TABLE base_tab; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q new file mode 100644 index 0000000000000..d077981d02ad0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q @@ -0,0 +1,24 @@ +-- base table with null data +DROP TABLE IF EXISTS base_tab; +CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab; +DESCRIBE EXTENDED base_tab; + +-- table with non-default null format +DROP TABLE IF EXISTS null_tab3; +EXPLAIN CREATE TABLE null_tab3 ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull' + AS SELECT a, b FROM base_tab; +CREATE TABLE null_tab3 ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull' + AS SELECT a, b FROM base_tab; +DESCRIBE EXTENDED null_tab3; +SHOW CREATE TABLE null_tab3; + +dfs -cat ${system:test.warehouse.dir}/null_tab3/*; +SELECT * FROM null_tab3; +-- alter the null format and verify that the old null format is no longer in effect +ALTER TABLE null_tab3 SET SERDEPROPERTIES ( 'serialization.null.format'='foo'); +SELECT * FROM null_tab3; + + +DROP TABLE null_tab3; +DROP TABLE base_tab; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q new file mode 100644 index 0000000000000..d29863839f728 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q @@ -0,0 +1,21 @@ +-- base table with null data +DROP TABLE IF EXISTS base_tab; +CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab; +DESCRIBE EXTENDED base_tab; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/hive_test/nullformat/tmp; +dfs -rmr ${system:test.tmp.dir}/hive_test/nullformat/*; +INSERT OVERWRITE LOCAL DIRECTORY '${system:test.tmp.dir}/hive_test/nullformat' + ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull' SELECT a,b FROM base_tab; +dfs -cat ${system:test.tmp.dir}/hive_test/nullformat/000000_0; + +-- load the exported data back into a table with same null format and verify null values +DROP TABLE IF EXISTS null_tab2; +CREATE TABLE null_tab2(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull'; +LOAD DATA LOCAL INPATH '${system:test.tmp.dir}/hive_test/nullformat/000000_0' INTO TABLE null_tab2; +SELECT * FROM null_tab2; + + +dfs -rmr ${system:test.tmp.dir}/hive_test/nullformat; +DROP TABLE base_tab; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q index a5bc9ff1f36ff..19e5b10786fc0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q @@ -1,28 +1,28 @@ CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08'); explain select count(1) from tstparttbl; select count(1) from tstparttbl; CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08'); explain select count(1) from tstparttbl2; select count(1) from tstparttbl2; DROP TABLE tstparttbl; CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08'); explain select count(1) from tstparttbl; select count(1) from tstparttbl; DROP TABLE tstparttbl2; CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08'); explain select count(1) from tstparttbl2; select count(1) from tstparttbl2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q index 12773b6159a5d..b4b68fb8b693a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q @@ -1,10 +1,10 @@ CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2009-04-09'); CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl2 PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl2 PARTITION (ds='2009-04-09'); explain select u.* from diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q index 95c9e1df37679..11f4a7a78f813 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q @@ -1,7 +1,7 @@ CREATE TABLE nullscript(KEY STRING, VALUE STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE nullscript; -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE nullscript; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE nullscript; +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE nullscript; explain select transform(key) using 'cat' as key1 from nullscript; select transform(key) using 'cat' as key1 from nullscript; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q index 7f858d3e6f136..d51c2107e1586 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT null + 7, 1.0 - null, null + null, CAST(21 AS BIGINT) % CAST(5 AS TINYINT), CAST(21 AS BIGINT) % CAST(21 AS BIGINT), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q index b685ae6c4e53f..ec9e8076fd533 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; select 1.0 < 2.0 from src limit 1; select 2.0 < 2.0 from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q deleted file mode 100644 index 5993041405ed8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q +++ /dev/null @@ -1,6 +0,0 @@ -SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.OptrStatGroupByHook; -SET hive.exec.mode.local.auto=false; -SET hive.task.progress=true; --- This test executes the OptrStatGroupBy hook which prints the optr level --- stats of GROUPBY optr present is the plan of below query -SELECT count(1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q new file mode 100644 index 0000000000000..915f4f0d71e63 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q @@ -0,0 +1,179 @@ +CREATE TABLE orc_create_people_staging ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp, + state string); + +LOAD DATA LOCAL INPATH '../../data/files/orc_create_people.txt' OVERWRITE INTO TABLE orc_create_people_staging; + +set hive.exec.dynamic.partition.mode=nonstrict; + +set hive.stats.autogather=false; +-- non-partitioned table +-- partial scan gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp, + state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people SELECT * FROM orc_create_people_staging ORDER BY id; + +set hive.stats.autogather = true; +analyze table orc_create_people compute statistics partialscan; + +desc formatted orc_create_people; + +drop table orc_create_people; + +-- auto stats gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp, + state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people SELECT * FROM orc_create_people_staging ORDER BY id; + +desc formatted orc_create_people; + +drop table orc_create_people; + +set hive.stats.autogather=false; +-- partitioned table +-- partial scan gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +set hive.stats.autogather = true; +analyze table orc_create_people partition(state) compute statistics partialscan; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="Or"); + +drop table orc_create_people; + +-- auto stats gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="Or"); + +drop table orc_create_people; + +set hive.stats.autogather=false; +-- partitioned and bucketed table +-- partial scan gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +clustered by (first_name) +sorted by (last_name) +into 4 buckets +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +set hive.stats.autogather = true; +analyze table orc_create_people partition(state) compute statistics partialscan; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="Or"); + +drop table orc_create_people; + +-- auto stats gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +clustered by (first_name) +sorted by (last_name) +into 4 buckets +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="Or"); + +drop table orc_create_people; + +set hive.stats.autogather=false; +-- create table with partitions containing text and ORC files. +-- ORC files implements StatsProvidingRecordReader but text files does not. +-- So the partition containing text file should not have statistics. +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +-- set the table to text format +ALTER TABLE orc_create_people SET SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; +ALTER TABLE orc_create_people SET FILEFORMAT TEXTFILE; + +-- load the text data into a new partition +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE orc_create_people PARTITION(state="OH"); + +-- set the table back to orc +ALTER TABLE orc_create_people SET SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'; +ALTER TABLE orc_create_people SET FILEFORMAT ORC; + +set hive.stats.autogather = true; +analyze table orc_create_people partition(state) compute statistics noscan; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="OH"); + +drop table orc_create_people; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q index 6aca5486445c8..a82c1a55d5f6c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q @@ -40,7 +40,7 @@ set hive.default.fileformat=orc; CREATE TABLE orc_create (key INT, value STRING) PARTITIONED BY (ds string); -set hive.default.fileformat=text; +set hive.default.fileformat=TextFile; DESCRIBE FORMATTED orc_create; @@ -53,7 +53,7 @@ CREATE TABLE orc_create_complex ( DESCRIBE FORMATTED orc_create_complex; -LOAD DATA LOCAL INPATH '../data/files/orc_create.txt' OVERWRITE INTO TABLE orc_create_staging; +LOAD DATA LOCAL INPATH '../../data/files/orc_create.txt' OVERWRITE INTO TABLE orc_create_staging; SELECT * from orc_create_staging; @@ -70,16 +70,20 @@ CREATE TABLE orc_create_people_staging ( first_name string, last_name string, address string, + salary decimal, + start_date timestamp, state string); -LOAD DATA LOCAL INPATH '../data/files/orc_create_people.txt' +LOAD DATA LOCAL INPATH '../../data/files/orc_create_people.txt' OVERWRITE INTO TABLE orc_create_people_staging; CREATE TABLE orc_create_people ( id int, first_name string, last_name string, - address string) + address string, + salary decimal, + start_date timestamp) PARTITIONED BY (state string) STORED AS orc; @@ -92,9 +96,26 @@ SET hive.optimize.index.filter=true; -- test predicate push down with partition pruning SELECT COUNT(*) FROM orc_create_people where id < 10 and state = 'Ca'; +-- test predicate push down +SELECT COUNT(*) FROM orc_create_people where id = 50; +SELECT COUNT(*) FROM orc_create_people where id between 10 and 20; +SELECT COUNT(*) FROM orc_create_people where id > 10 and id < 100; +SELECT COUNT(*) FROM orc_create_people where (id + 1) = 20; +SELECT COUNT(*) FROM orc_create_people where (id + 10) < 200; +SELECT COUNT(*) FROM orc_create_people where id < 30 or first_name = "Rafael"; +SELECT COUNT(*) FROM orc_create_people + where length(substr(first_name, 1, 2)) <= 2 and last_name like '%'; +SELECT COUNT(*) FROM orc_create_people where salary = 200.00; +SELECT COUNT(*) FROM orc_create_people WHERE start_date IS NULL; +SELECT COUNT(*) FROM orc_create_people WHERE YEAR(start_date) = 2014; + +-- test predicate push down with partition pruning +SELECT COUNT(*) FROM orc_create_people where salary = 200.00 and state = 'Ca'; + -- test predicate push down with no column projection SELECT id, first_name, last_name, address - FROM orc_create_people WHERE id > 90; + FROM orc_create_people WHERE id > 90 + ORDER BY id, first_name, last_name; DROP TABLE orc_create; DROP TABLE orc_create_complex; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q index f916012b8365e..a0eaab75fac3e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q @@ -19,7 +19,7 @@ SELECT * FROM test_orc; ALTER TABLE test_orc SET SERDEPROPERTIES ('orc.stripe.size' = '1'); CREATE TABLE src_thousand(key STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1kv2.cogroup.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1kv2.cogroup.txt' INTO TABLE src_thousand; set hive.exec.orc.dictionary.key.size.threshold=0.5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q index cbfd7b359f878..0c8861e41c4bb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q @@ -10,7 +10,7 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -- to another partition -- This can produce unexpected results with CombineHiveInputFormat -INSERT OVERWRITE TABLE test_orc PARTITION (part = '1') SELECT key FROM src LIMIT 5; +INSERT OVERWRITE TABLE test_orc PARTITION (part = '1') SELECT key FROM src tablesample (5 rows); ALTER TABLE test_orc ADD COLUMNS (cnt INT); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q new file mode 100644 index 0000000000000..f7e80a75b3ba6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q @@ -0,0 +1,11 @@ +-- Create a table with one column, write to it, then add an additional column +-- This can break reads + +CREATE TABLE test_orc (key STRING) +STORED AS ORC; + +INSERT OVERWRITE TABLE test_orc SELECT key FROM src LIMIT 5; + +ALTER TABLE test_orc ADD COLUMNS (value STRING); + +SELECT * FROM test_orc order by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q index 0ef57d18ccaf7..34cd6d47dfcd9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q @@ -3,13 +3,13 @@ ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; -INSERT OVERWRITE TABLE test_orc SELECT '' FROM src limit 10; +INSERT OVERWRITE TABLE test_orc SELECT '' FROM src tablesample (10 rows); -- Test reading a column which is just empty strings SELECT * FROM test_orc; -INSERT OVERWRITE TABLE test_orc SELECT IF (key % 3 = 0, key, '') FROM src limit 10; +INSERT OVERWRITE TABLE test_orc SELECT IF (key % 3 = 0, key, '') FROM src tablesample (10 rows); -- Test reading a column which has some empty strings diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q index 6685da7a82245..83c5a0505e57e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q @@ -10,7 +10,7 @@ ALTER TABLE test_orc SET SERDEPROPERTIES ('orc.row.index.stride' = '1000'); -- to last index stride are the same (there's only two index strides) CREATE TABLE src_null(a STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/nulls.txt' INTO TABLE src_null; +LOAD DATA LOCAL INPATH '../../data/files/nulls.txt' INTO TABLE src_null; INSERT OVERWRITE TABLE test_orc SELECT a FROM src_null; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q new file mode 100644 index 0000000000000..b81adf2af3e53 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q @@ -0,0 +1,32 @@ +create table if not exists alltypes ( + bo boolean, + ti tinyint, + si smallint, + i int, + bi bigint, + f float, + d double, + de decimal(10,3), + ts timestamp, + da date, + s string, + c char(5), + vc varchar(5), + m map, + l array, + st struct +) row format delimited fields terminated by '|' +collection items terminated by ',' +map keys terminated by ':' stored as textfile; + +create table alltypes_orc like alltypes; +alter table alltypes_orc set fileformat orc; + +load data local inpath '../../data/files/alltypes2.txt' overwrite into table alltypes; + +insert overwrite table alltypes_orc select * from alltypes; + +select min(bo), max(bo), min(ti), max(ti), min(si), max(si), min(i), max(i), min(bi), max(bi), min(f), max(f), min(d), max(d), min(de), max(de), min(ts), max(ts), min(da), max(da), min(s), max(s), min(c), max(c), min(vc), max(vc) from alltypes; + +select min(bo), max(bo), min(ti), max(ti), min(si), max(si), min(i), max(i), min(bi), max(bi), min(f), max(f), min(d), max(d), min(de), max(de), min(ts), max(ts), min(da), max(da), min(s), max(s), min(c), max(c), min(vc), max(vc) from alltypes_orc; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q new file mode 100644 index 0000000000000..1f5f54ae19ee8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q @@ -0,0 +1,76 @@ +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET mapred.min.split.size=1000; +SET mapred.max.split.size=5000; + +create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); + +insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl; + +set hive.optimize.index.filter=false; + +-- char data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests) +select sum(hash(*)) from newtypesorc where c="apple"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c="apple"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c!="apple"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c!="apple"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c<"hello"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c<"hello"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c<="hello"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c<="hello"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c="apple "; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c="apple "; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c in ("apple", "carrot"); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c in ("apple", "carrot"); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c in ("apple", "hello"); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c in ("apple", "hello"); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c in ("carrot"); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c in ("carrot"); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c between "apple" and "carrot"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c between "apple" and "carrot"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c between "apple" and "zombie"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c between "apple" and "zombie"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c between "carrot" and "carrot1"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c between "carrot" and "carrot1"; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q new file mode 100644 index 0000000000000..c34be867e484f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q @@ -0,0 +1,97 @@ +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET mapred.min.split.size=1000; +SET mapred.max.split.size=5000; + +create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); + +insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl; + +-- date data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests) +select sum(hash(*)) from newtypesorc where da='1970-02-20'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da='1970-02-20'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as date); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as date); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as varchar(20)); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as varchar(20)); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da!='1970-02-20'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da!='1970-02-20'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<'1970-02-27'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<'1970-02-27'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<'1970-02-29'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<'1970-02-29'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<'1970-02-15'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<'1970-02-15'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<='1970-02-20'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<='1970-02-20'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<='1970-02-27'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<='1970-02-27'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-27' as date)); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-27' as date)); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-20' as date), cast('1970-02-27' as date)); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-20' as date), cast('1970-02-27' as date)); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-22' as date)); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-22' as date)); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-22'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-22'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-28'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-28'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da between '1970-02-18' and '1970-02-19'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da between '1970-02-18' and '1970-02-19'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q new file mode 100644 index 0000000000000..a93590eacca01 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q @@ -0,0 +1,151 @@ +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET mapred.min.split.size=1000; +SET mapred.max.split.size=5000; + +create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); + +insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl; + +-- decimal data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests) +select sum(hash(*)) from newtypesorc where d=0.22; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d=0.22; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d='0.22'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d='0.22'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d=cast('0.22' as float); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d=cast('0.22' as float); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d!=0.22; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d!=0.22; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d!='0.22'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d!='0.22'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d!=cast('0.22' as float); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d!=cast('0.22' as float); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d<11.22; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d<11.22; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d<'11.22'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d<'11.22'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d, + lst ARRAY, + strct STRUCT +) ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|' +COLLECTION ITEMS TERMINATED BY ',' +MAP KEYS TERMINATED BY ':'; + +CREATE TABLE parquet_create ( + id int, + str string, + mp MAP, + lst ARRAY, + strct STRUCT +) STORED AS PARQUET; + +DESCRIBE FORMATTED parquet_create; + +LOAD DATA LOCAL INPATH '../../data/files/parquet_create.txt' OVERWRITE INTO TABLE parquet_create_staging; + +SELECT * FROM parquet_create_staging; + +INSERT OVERWRITE TABLE parquet_create SELECT * FROM parquet_create_staging; + +SELECT * FROM parquet_create group by id; +SELECT id, count(0) FROM parquet_create group by id; +SELECT str from parquet_create; +SELECT mp from parquet_create; +SELECT lst from parquet_create; +SELECT strct from parquet_create; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q new file mode 100644 index 0000000000000..652aef1b2ba53 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q @@ -0,0 +1,24 @@ +drop table staging; +drop table parquet_ctas; +drop table parquet_ctas_advanced; +drop table parquet_ctas_alias; +drop table parquet_ctas_mixed; + +create table staging (key int, value string) stored as textfile; +insert into table staging select * from src order by key limit 10; + +create table parquet_ctas stored as parquet as select * from staging; +describe parquet_ctas; +select * from parquet_ctas; + +create table parquet_ctas_advanced stored as parquet as select key+1,concat(value,"value") from staging; +describe parquet_ctas_advanced; +select * from parquet_ctas_advanced; + +create table parquet_ctas_alias stored as parquet as select key+1 as mykey,concat(value,"value") as myvalue from staging; +describe parquet_ctas_alias; +select * from parquet_ctas_alias; + +create table parquet_ctas_mixed stored as parquet as select key,key+1,concat(value,"value") as myvalue from staging; +describe parquet_ctas_mixed; +select * from parquet_ctas_mixed; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q new file mode 100644 index 0000000000000..5d4f68ea43723 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q @@ -0,0 +1,34 @@ +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +DROP TABLE parquet_partitioned_staging; +DROP TABLE parquet_partitioned; + +CREATE TABLE parquet_partitioned_staging ( + id int, + str string, + part string +) ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|'; + +CREATE TABLE parquet_partitioned ( + id int, + str string +) PARTITIONED BY (part string) +STORED AS PARQUET; + +DESCRIBE FORMATTED parquet_partitioned; + +LOAD DATA LOCAL INPATH '../../data/files/parquet_partitioned.txt' OVERWRITE INTO TABLE parquet_partitioned_staging; + +SELECT * FROM parquet_partitioned_staging; + +INSERT OVERWRITE TABLE parquet_partitioned PARTITION (part) SELECT * FROM parquet_partitioned_staging; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT * FROM parquet_partitioned ORDER BY id, str; +SELECT part, COUNT(0) FROM parquet_partitioned GROUP BY part; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +SELECT * FROM parquet_partitioned ORDER BY id, str; +SELECT part, COUNT(0) FROM parquet_partitioned GROUP BY part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q new file mode 100644 index 0000000000000..5d6333c934b74 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q @@ -0,0 +1,38 @@ +DROP TABLE parquet_types_staging; +DROP TABLE parquet_types; + +CREATE TABLE parquet_types_staging ( + cint int, + ctinyint tinyint, + csmallint smallint, + cfloat float, + cdouble double, + cstring1 string +) ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|'; + +CREATE TABLE parquet_types ( + cint int, + ctinyint tinyint, + csmallint smallint, + cfloat float, + cdouble double, + cstring1 string +) STORED AS PARQUET; + +LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_types_staging; + +INSERT OVERWRITE TABLE parquet_types SELECT * FROM parquet_types_staging; + +SELECT * FROM parquet_types; + +SELECT ctinyint, + MAX(cint), + MIN(csmallint), + COUNT(cstring1), + AVG(cfloat), + STDDEV_POP(cdouble) +FROM parquet_types +GROUP BY ctinyint +ORDER BY ctinyint +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q index b7f8c64d4261f..03a5760e690e9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q @@ -1,7 +1,7 @@ create table test1(col1 string) partitioned by (partitionId int); insert overwrite table test1 partition (partitionId=1) - select key from src limit 10; + select key from src tablesample (10 rows); FROM ( FROM test1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q index 8738afdfa099a..70a7b252154d1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q @@ -1,45 +1,58 @@ drop table partition_date_1; -create table partition_date_1 (key string, value string) partitioned by (dt date, region int); +create table partition_date_1 (key string, value string) partitioned by (dt date, region string); + +insert overwrite table partition_date_1 partition(dt='2000-01-01', region= '1') + select * from src tablesample (10 rows); +insert overwrite table partition_date_1 partition(dt='2000-01-01', region= '2') + select * from src tablesample (5 rows); +insert overwrite table partition_date_1 partition(dt='2013-12-10', region= '2020-20-20') + select * from src tablesample (5 rows); +insert overwrite table partition_date_1 partition(dt='2013-08-08', region= '1') + select * from src tablesample (20 rows); +insert overwrite table partition_date_1 partition(dt='2013-08-08', region= '10') + select * from src tablesample (11 rows); -insert overwrite table partition_date_1 partition(dt='2000-01-01', region=1) - select * from src limit 10; -insert overwrite table partition_date_1 partition(dt='2000-01-01', region=2) - select * from src limit 5; -insert overwrite table partition_date_1 partition(dt='2013-08-08', region=1) - select * from src limit 20; -insert overwrite table partition_date_1 partition(dt='2013-08-08', region=10) - select * from src limit 11; select distinct dt from partition_date_1; -select * from partition_date_1 where dt = '2000-01-01' and region = 2 order by key,value; +select * from partition_date_1 where dt = '2000-01-01' and region = '2' order by key,value; -- 15 select count(*) from partition_date_1 where dt = date '2000-01-01'; -- 15. Also try with string value in predicate select count(*) from partition_date_1 where dt = '2000-01-01'; -- 5 -select count(*) from partition_date_1 where dt = date '2000-01-01' and region = 2; +select count(*) from partition_date_1 where dt = date '2000-01-01' and region = '2'; -- 11 -select count(*) from partition_date_1 where dt = date '2013-08-08' and region = 10; +select count(*) from partition_date_1 where dt = date '2013-08-08' and region = '10'; -- 30 -select count(*) from partition_date_1 where region = 1; +select count(*) from partition_date_1 where region = '1'; -- 0 -select count(*) from partition_date_1 where dt = date '2000-01-01' and region = 3; +select count(*) from partition_date_1 where dt = date '2000-01-01' and region = '3'; -- 0 select count(*) from partition_date_1 where dt = date '1999-01-01'; -- Try other comparison operations -- 20 -select count(*) from partition_date_1 where dt > date '2000-01-01' and region = 1; +select count(*) from partition_date_1 where dt > date '2000-01-01' and region = '1'; -- 10 -select count(*) from partition_date_1 where dt < date '2000-01-02' and region = 1; +select count(*) from partition_date_1 where dt < date '2000-01-02' and region = '1'; -- 20 -select count(*) from partition_date_1 where dt >= date '2000-01-02' and region = 1; +select count(*) from partition_date_1 where dt >= date '2000-01-02' and region = '1'; -- 10 -select count(*) from partition_date_1 where dt <= date '2000-01-01' and region = 1; +select count(*) from partition_date_1 where dt <= date '2000-01-01' and region = '1'; -- 20 -select count(*) from partition_date_1 where dt <> date '2000-01-01' and region = 1; +select count(*) from partition_date_1 where dt <> date '2000-01-01' and region = '1'; +-- 10 +select count(*) from partition_date_1 where dt between date '1999-12-30' and date '2000-01-03' and region = '1'; + + +-- Try a string key with date-like strings + +-- 5 +select count(*) from partition_date_1 where region = '2020-20-20'; +-- 5 +select count(*) from partition_date_1 where region > '2010-01-01'; drop table partition_date_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q index 9b84b59608503..c932ed1023637 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q @@ -3,7 +3,7 @@ drop table partition_date2_1; create table partition_date2_1 (key string, value string) partitioned by (dt date, region int); -- test date literal syntax -from (select * from src limit 1) x +from (select * from src tablesample (1 rows)) x insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=1) select * insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=2) select * insert overwrite table partition_date2_1 partition(dt=date '1999-01-01', region=2) select *; @@ -13,7 +13,7 @@ select * from partition_date2_1; -- insert overwrite insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=2) - select 'changed_key', 'changed_value' from src limit 2; + select 'changed_key', 'changed_value' from src tablesample (2 rows); select * from partition_date2_1; -- truncate @@ -41,7 +41,7 @@ alter table partition_date2_1 partition(dt=date '1980-01-02', region=3) describe extended partition_date2_1 partition(dt=date '1980-01-02', region=3); insert overwrite table partition_date2_1 partition(dt=date '1980-01-02', region=3) - select * from src limit 2; + select * from src tablesample (2 rows); select * from partition_date2_1 order by key,value,dt,region; -- alter table set location diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q index ba193cd51a26d..a8381a4200f0a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q @@ -1,9 +1,9 @@ create table sc as select * -from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 +from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows) union all - select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows) union all - select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s; + select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s; create table sc_part (key string) partitioned by (ts string) stored as rcfile; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q index 81344334dfe01..b0b1ff4db6074 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q @@ -1,9 +1,9 @@ create table sc as select * -from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 +from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows) union all - select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows) union all - select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s; + select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s; create table sc_part (key string) partitioned by (ts string) stored as rcfile; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q index 7f1accadac6ea..c9bca99b9cdf6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q @@ -2,14 +2,14 @@ set hive.typecheck.on.insert = true; -- begin part(string, string) pass(string, int) CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day string) stored as textfile; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day=2); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day=2); select * from tab1; drop table tab1; -- begin part(string, int) pass(string, string) CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day int) stored as textfile; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2'); select * from tab1; drop table tab1; @@ -17,7 +17,7 @@ drop table tab1; -- begin part(string, date) pass(string, date) create table tab1 (id1 int, id2 string) PARTITIONED BY(month string,day date) stored as textfile; alter table tab1 add partition (month='June', day='2008-01-01'); -LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2008-01-01'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2008-01-01'); select id1, id2, day from tab1 where day='2008-01-01'; drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q index d700b1cbf8566..22aadd3b5359f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q @@ -3,13 +3,13 @@ drop table partition_varchar_1; create table partition_varchar_1 (key string, value varchar(20)) partitioned by (dt varchar(10), region int); insert overwrite table partition_varchar_1 partition(dt='2000-01-01', region=1) - select * from src limit 10; + select * from src tablesample (10 rows); insert overwrite table partition_varchar_1 partition(dt='2000-01-01', region=2) - select * from src limit 5; + select * from src tablesample (5 rows); insert overwrite table partition_varchar_1 partition(dt='2013-08-08', region=1) - select * from src limit 20; + select * from src tablesample (20 rows); insert overwrite table partition_varchar_1 partition(dt='2013-08-08', region=10) - select * from src limit 11; + select * from src tablesample (11 rows); select distinct dt from partition_varchar_1; select * from partition_varchar_1 where dt = '2000-01-01' and region = 2 order by key,value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q new file mode 100644 index 0000000000000..92cb742f15011 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q @@ -0,0 +1,10 @@ +drop table partition_varchar_2; + +create table partition_varchar_2 (key string, value varchar(20)) partitioned by (dt varchar(15), region int); + +insert overwrite table partition_varchar_2 partition(dt='2000-01-01', region=1) + select * from src order by key limit 1; + +select * from partition_varchar_2 where cast(dt as varchar(10)) = '2000-01-01'; + +drop table partition_varchar_2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q index e9b574c1ca380..3cf488fb0337f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q @@ -3,9 +3,9 @@ -- CustomSerDe(1, 2, 3) irrespective of the inserted values DROP TABLE PW17; -ADD JAR ../build/ql/test/test-serdes.jar; +ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-custom-serde/${system:hive.version}/hive-it-custom-serde-${system:hive.version}.jar; CREATE TABLE PW17(USER STRING, COMPLEXDT ARRAY) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1'; -LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17 PARTITION (YEAR='1'); +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17 PARTITION (YEAR='1'); ALTER TABLE PW17 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe2'; ALTER TABLE PW17 SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1'; -- Without the fix HIVE-5199, will throw cast exception via FetchOperator @@ -14,13 +14,13 @@ SELECT * FROM PW17; -- Test for non-parititioned table. DROP TABLE PW17_2; CREATE TABLE PW17_2(USER STRING, COMPLEXDT ARRAY) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1'; -LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_2; +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_2; -- Without the fix HIVE-5199, will throw cast exception via MapOperator SELECT COUNT(*) FROM PW17_2; DROP TABLE PW17_3; CREATE TABLE PW17_3(USER STRING, COMPLEXDT ARRAY >) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3'; -LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_3 PARTITION (YEAR='1'); +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_3 PARTITION (YEAR='1'); ALTER TABLE PW17_3 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe2'; ALTER TABLE PW17_3 SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3'; -- Without the fix HIVE-5285, will throw cast exception via FetchOperator @@ -28,7 +28,7 @@ SELECT * FROM PW17; DROP TABLE PW17_4; CREATE TABLE PW17_4(USER STRING, COMPLEXDT ARRAY >) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3'; -LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_4; +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_4; -- Without the fix HIVE-5285, will throw cast exception via MapOperator SELECT COUNT(*) FROM PW17_4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q new file mode 100644 index 0000000000000..40ed2585f5122 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q @@ -0,0 +1,19 @@ +-- HIVE-5202 : Tests for SettableUnionObjectInspectors +-- CustomSerDe(4,5) are used here. +-- The final results should be all NULL columns deserialized using +-- CustomSerDe(4, 5) irrespective of the inserted values + +DROP TABLE PW18; +ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-custom-serde/${system:hive.version}/hive-it-custom-serde-${system:hive.version}.jar; +CREATE TABLE PW18(USER STRING, COMPLEXDT UNIONTYPE) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe5'; +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW18 PARTITION (YEAR='1'); +ALTER TABLE PW18 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe4'; +-- Without the fix HIVE-5202, will throw unsupported data type exception. +SELECT * FROM PW18; + +-- Test for non-parititioned table. +DROP TABLE PW18_2; +CREATE TABLE PW18_2(USER STRING, COMPLEXDT UNIONTYPE) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe5'; +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW18_2; +-- Without the fix HIVE-5202, will throw unsupported data type exception +SELECT COUNT(*) FROM PW18_2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q index 09a39ae4e4476..3be0ff23b8124 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q @@ -127,7 +127,7 @@ create table ab(strct struct) row format delimited fields terminated by '\t' collection items terminated by '\001'; -load data local inpath '../data/files/kv1.txt' +load data local inpath '../../data/files/kv1.txt' overwrite into table ab; -- Create partitioned table with struct data: diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q new file mode 100644 index 0000000000000..475d45c19ff0c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q @@ -0,0 +1,22 @@ +create table dual(a string); + +set hive.optimize.ppd=true; +drop table if exists test_tbl ; + +create table test_tbl (id string,name string); + +insert into table test_tbl +select 'a','b' from dual; + +explain +select t2.* +from +(select id,name from (select id,name from test_tbl) t1 sort by id) t2 +join test_tbl t3 on (t2.id=t3.id ) +where t2.name='c' and t3.id='a'; + +select t2.* +from +(select id,name from (select id,name from test_tbl) t1 sort by id) t2 +join test_tbl t3 on (t2.id=t3.id ) +where t2.name='c' and t3.id='a'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q index a802df1b98819..06fe7ce580c84 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q @@ -10,18 +10,18 @@ FROM src a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; FROM src a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; SELECT mi1.* FROM mi1; SELECT mi2.* FROM mi2; SELECT mi3.* FROM mi3; -dfs -cat ../build/ql/test/data/warehouse/mi4.out/*; +dfs -cat ${system:test.warehouse.dir}/mi4.out/*; set hive.ppd.remove.duplicatefilters=true; @@ -31,15 +31,15 @@ FROM src a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; FROM src a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; SELECT mi1.* FROM mi1; SELECT mi2.* FROM mi2; SELECT mi3.* FROM mi3; -dfs -cat ../build/ql/test/data/warehouse/mi4.out/*; +dfs -cat ${system:test.warehouse.dir}/mi4.out/*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q index 65a498d021f77..530ef9c4d849a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q @@ -36,3 +36,12 @@ FROM ( CLUSTER BY tkey ) tmap SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +-- test described in HIVE-4598 + +EXPLAIN +FROM ( + FROM ( SELECT * FROM src ) mapout REDUCE * USING 'cat' AS x,y +) reduced +insert overwrite local directory '/tmp/a' select * where x='a' or x='b' +insert overwrite local directory '/tmp/b' select * where x='c' or x='d'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q new file mode 100644 index 0000000000000..d90532cfa4bcd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q @@ -0,0 +1,12 @@ +explain +SELECT value from ( + select explode(array(key, value)) as (value) from ( + select * FROM src WHERE key > 400 + ) A +) B WHERE value < 450; + +SELECT value from ( + select explode(array(key, value)) as (value) from ( + select * FROM src WHERE key > 400 + ) A +) B WHERE value < 450; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q index d635e2d171469..a7606c5a5f2a5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q @@ -5,26 +5,26 @@ drop view v; create table t1_new (key string, value string) partitioned by (ds string); insert overwrite table t1_new partition (ds = '2011-10-15') -select 'key1', 'value1' from src limit 1; +select 'key1', 'value1' from src tablesample (1 rows); insert overwrite table t1_new partition (ds = '2011-10-16') -select 'key2', 'value2' from src limit 1; +select 'key2', 'value2' from src tablesample (1 rows); create table t1_old (keymap string, value string) partitioned by (ds string); insert overwrite table t1_old partition (ds = '2011-10-13') -select 'keymap3', 'value3' from src limit 1; +select 'keymap3', 'value3' from src tablesample (1 rows); insert overwrite table t1_old partition (ds = '2011-10-14') -select 'keymap4', 'value4' from src limit 1; +select 'keymap4', 'value4' from src tablesample (1 rows); create table t1_mapping (key string, keymap string) partitioned by (ds string); insert overwrite table t1_mapping partition (ds = '2011-10-13') -select 'key3', 'keymap3' from src limit 1; +select 'key3', 'keymap3' from src tablesample (1 rows); insert overwrite table t1_mapping partition (ds = '2011-10-14') -select 'key4', 'keymap4' from src limit 1; +select 'key4', 'keymap4' from src tablesample (1 rows); create view t1 partitioned on (ds) as diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q index 860dd631ce10a..440005fdee951 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + create table ppr_test (key string) partitioned by (ds string); alter table ppr_test add partition (ds = '1234'); @@ -9,14 +11,14 @@ alter table ppr_test add partition (ds = '12:4'); alter table ppr_test add partition (ds = '12%4'); alter table ppr_test add partition (ds = '12*4'); -insert overwrite table ppr_test partition(ds = '1234') select * from (select '1234' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '1224') select * from (select '1224' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '1214') select * from (select '1214' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12+4') select * from (select '12+4' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12.4') select * from (select '12.4' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12:4') select * from (select '12:4' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12%4') select * from (select '12%4' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12*4') select * from (select '12*4' from src limit 1 union all select 'abcd' from src limit 1) s; +insert overwrite table ppr_test partition(ds = '1234') select * from (select '1234' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '1224') select * from (select '1224' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '1214') select * from (select '1214' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12+4') select * from (select '12+4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12.4') select * from (select '12.4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12:4') select * from (select '12:4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12%4') select * from (select '12%4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12*4') select * from (select '12*4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; select * from ppr_test where ds = '1234' order by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q index 67c0da0dfc591..8c6090653811b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q @@ -1,24 +1,26 @@ +set hive.fetch.task.conversion=more; + create table ppr_test (key string) partitioned by (ds string); -insert overwrite table ppr_test partition(ds='2') select '2' from src limit 1; -insert overwrite table ppr_test partition(ds='22') select '22' from src limit 1; +insert overwrite table ppr_test partition(ds='2') select '2' from src tablesample (1 rows); +insert overwrite table ppr_test partition(ds='22') select '22' from src tablesample (1 rows); select * from ppr_test where ds = '2'; select * from ppr_test where ds = '22'; create table ppr_test2 (key string) partitioned by (ds string, s string); -insert overwrite table ppr_test2 partition(ds='1', s='2') select '1' from src limit 1; -insert overwrite table ppr_test2 partition(ds='2', s='1') select '2' from src limit 1; +insert overwrite table ppr_test2 partition(ds='1', s='2') select '1' from src tablesample (1 rows); +insert overwrite table ppr_test2 partition(ds='2', s='1') select '2' from src tablesample (1 rows); select * from ppr_test2 where s = '1'; select * from ppr_test2 where ds = '1'; create table ppr_test3 (key string) partitioned by (col string, ol string, l string); -insert overwrite table ppr_test3 partition(col='1', ol='2', l = '3') select '1' from src limit 1; -insert overwrite table ppr_test3 partition(col='1', ol='1', l = '2') select '2' from src limit 1; -insert overwrite table ppr_test3 partition(col='1', ol='2', l = '1') select '3' from src limit 1; +insert overwrite table ppr_test3 partition(col='1', ol='2', l = '3') select '1' from src tablesample (1 rows); +insert overwrite table ppr_test3 partition(col='1', ol='1', l = '2') select '2' from src tablesample (1 rows); +insert overwrite table ppr_test3 partition(col='1', ol='2', l = '1') select '3' from src tablesample (1 rows); select * from ppr_test3 where l = '1'; select * from ppr_test3 where l = '2'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q index ad908a02ad181..22ee92634d123 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q @@ -2,7 +2,7 @@ set hive.heartbeat.interval=5; CREATE TABLE PROGRESS_1(key int, value string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv6.txt' INTO TABLE PROGRESS_1; +LOAD DATA LOCAL INPATH '../../data/files/kv6.txt' INTO TABLE PROGRESS_1; select count(1) from PROGRESS_1 t1 join PROGRESS_1 t2 on t1.key=t2.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q index eea5415d682a1..d56b4123554c4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; --1. test1 select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q index 03f435e4539e2..9799534ff4a55 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q @@ -9,11 +9,11 @@ CREATE TABLE part( p_type STRING, p_size INT, p_container STRING, - p_retailprice DECIMAL, + p_retailprice DECIMAL(6,2), p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. aggregate functions with decimal type diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q index 885c3b3d43e17..4fe9710d0d247 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. testNoPTFNoWindowing select p_mfgr, p_name, p_size diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q index 72eeb104d5298..0cde350f73693 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q @@ -10,7 +10,7 @@ ARR_DELAY float, FL_NUM string ); -LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; -- 1. basic Matchpath test select origin_city_name, fl_num, year, month, day_of_month, sz, tpath diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q index 535a233a9e3c4..a68c578848dcb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q @@ -12,7 +12,7 @@ CREATE TABLE part_rc( p_comment STRING ) STORED AS RCFILE ; -LOAD DATA LOCAL INPATH '../data/files/part.rc' overwrite into table part_rc; +LOAD DATA LOCAL INPATH '../../data/files/part.rc' overwrite into table part_rc; -- testWindowingPTFWithPartRC select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q index a2140cd049f2d..4b508e9df0449 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q @@ -10,7 +10,7 @@ ARR_DELAY float, FL_NUM string ); -LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; create temporary function matchpathtest as 'org.apache.hadoop.hive.ql.udf.ptf.MatchPath$MatchPathResolver'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q index 4aa8ce11bec9c..c5d65f0efa211 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q @@ -12,7 +12,7 @@ CREATE TABLE part_seq( p_comment STRING ) STORED AS SEQUENCEFILE ; -LOAD DATA LOCAL INPATH '../data/files/part.seq' overwrite into table part_seq; +LOAD DATA LOCAL INPATH '../../data/files/part.seq' overwrite into table part_seq; -- testWindowingPTFWithPartSeqFile select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q index f198baa6e4c5b..57e8cc673cace 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q @@ -1,3 +1,5 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; DROP TABLE lineitem; CREATE TABLE lineitem (L_ORDERKEY INT, @@ -19,7 +21,7 @@ CREATE TABLE lineitem (L_ORDERKEY INT, ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; -LOAD DATA LOCAL INPATH '../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem; +LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem; CREATE INDEX lineitem_lshipdate_idx ON TABLE lineitem(l_shipdate) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(l_shipdate)"); ALTER INDEX lineitem_lshipdate_idx ON lineitem REBUILD; @@ -156,7 +158,7 @@ DROP INDEX tbl_part_index on tblpart; DROP TABLE tblpart; CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; -LOAD DATA LOCAL INPATH '../data/files/tbl.txt' OVERWRITE INTO TABLE tbl; +LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl; CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)"); ALTER INDEX tbl_key_idx ON tbl REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q index 65b9f8776d192..c93902ab3e39c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT 'abc', "abc", diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q new file mode 100644 index 0000000000000..a34a25af4bb96 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q @@ -0,0 +1,21 @@ + +set hive.support.quoted.identifiers=column; + +create table src_b3(`x+1` string, `!@#$%^&*()_q` string) ; + +alter table src_b3 +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +; + + +-- alter partition +create table src_p3(`x+1` string, `y&y` string) partitioned by (`!@#$%^&*()_q` string); + +insert overwrite table src_p3 partition(`!@#$%^&*()_q`='a') select * from src; +show partitions src_p3; + +alter table src_p3 add if not exists partition(`!@#$%^&*()_q`='b'); +show partitions src_p3; + +alter table src_p3 partition(`!@#$%^&*()_q`='b') rename to partition(`!@#$%^&*()_q`='c'); +show partitions src_p3; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q new file mode 100644 index 0000000000000..680868e549cee --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q @@ -0,0 +1,34 @@ + +set hive.support.quoted.identifiers=column; + +-- basic +create table t1(`x+1` string, `y&y` string, `!@#$%^&*()_q` string); +describe t1; +select `x+1`, `y&y`, `!@#$%^&*()_q` from t1; +explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1; +explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1 where `!@#$%^&*()_q` = '1'; +explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = '1'; +explain select `x+1`, `y&y`, `!@#$%^&*()_q`, rank() over(partition by `!@#$%^&*()_q` order by `y&y`) +from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = '1'; + +-- case insensitive +explain select `X+1`, `Y&y`, `!@#$%^&*()_Q`, rank() over(partition by `!@#$%^&*()_q` order by `y&y`) +from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&Y`, `!@#$%^&*()_q` having `!@#$%^&*()_Q` = '1'; + + +-- escaped back ticks +create table t4(`x+1``` string, `y&y` string); +describe t4; +insert into table t4 select * from src; +select `x+1```, `y&y`, rank() over(partition by `x+1``` order by `y&y`) +from t4 where `x+1``` = '10' group by `x+1```, `y&y` having `x+1``` = '10'; + +-- view +create view v1 as +select `x+1```, `y&y` +from t4 where `x+1``` < '200'; + +select `x+1```, `y&y`, rank() over(partition by `x+1``` order by `y&y`) +from v1 +group by `x+1```, `y&y` +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q new file mode 100644 index 0000000000000..e9416ae282228 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q @@ -0,0 +1,24 @@ + +set hive.support.quoted.identifiers=column; + + +create table src_p(`x+1` string, `y&y` string) partitioned by (`!@#$%^&*()_q` string); +insert overwrite table src_p partition(`!@#$%^&*()_q`='a') select * from src; + +show partitions src_p; + +explain select `x+1`, `y&y`, `!@#$%^&*()_q` +from src_p where `!@#$%^&*()_q` = 'a' and `x+1`='10' +group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = 'a' +; + +set hive.exec.dynamic.partition.mode=nonstrict +; + +create table src_p2(`x+1` string) partitioned by (`!@#$%^&*()_q` string); + +insert overwrite table src_p2 partition(`!@#$%^&*()_q`) +select key, value as `!@#$%^&*()_q` from src where key < '200' +; + +show partitions src_p2; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q new file mode 100644 index 0000000000000..5c959674117f1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q @@ -0,0 +1,26 @@ + +set hive.support.quoted.identifiers=column; + +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +set hive.optimize.skewjoin.compiletime = true; + +CREATE TABLE T1(`!@#$%^&*()_q` string, `y&y` string) +SKEWED BY (`!@#$%^&*()_q`) ON ((2)) STORED AS TEXTFILE +; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +CREATE TABLE T2(`!@#$%^&*()_q` string, `y&y` string) +SKEWED BY (`!@#$%^&*()_q`) ON ((2)) STORED AS TEXTFILE +; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T2; + +-- a simple join query with skew on both the tables on the join key +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a. `!@#$%^&*()_q` = b. `!@#$%^&*()_q` +; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q new file mode 100644 index 0000000000000..38d1b99c4b8e6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q @@ -0,0 +1,34 @@ + +set hive.support.quoted.identifiers=column; + + +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +create table src_b(`x+1` string, `!@#$%^&*()_q` string) +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +; + +insert overwrite table src_b +select * from src +; + +create table src_b2(`x+1` string, `!@#$%^&*()_q` string) +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +; + +insert overwrite table src_b2 +select * from src +; + +set hive.auto.convert.join=true; +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +set hive.auto.convert.sortmerge.join.to.mapjoin=false; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy = org.apache.hadoop.hive.ql.optimizer.TableSizeBasedBigTableSelectorForAutoSMJ; + +select a.`x+1`, a.`!@#$%^&*()_q`, b.`x+1`, b.`!@#$%^&*()_q` +from src_b a join src_b2 b on a.`!@#$%^&*()_q` = b.`!@#$%^&*()_q` +where a.`x+1` < '11' +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q new file mode 100644 index 0000000000000..d64e9cb9d524d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q @@ -0,0 +1,8 @@ +ADD 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 xyz(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' +STORED AS TEXTFILE +TBLPROPERTIES('columns'='valid_colname,invalid.colname') +; + +describe xyz; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q index 3e83e6693b276..df460c89aa896 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q @@ -1,7 +1,7 @@ set hive.map.aggr.hash.percentmemory = 0.3; set hive.mapred.local.mem = 256; -add file ../data/scripts/dumpdata_script.py; +add file ../../data/scripts/dumpdata_script.py; CREATE table columnTable_Bigdata (key STRING, value STRING) ROW FORMAT SERDE diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q index 9cfcee5a599c6..1c311fc478203 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q @@ -1,3 +1,5 @@ +set hive.support.quoted.identifiers=none; + EXPLAIN SELECT * FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q index 926601c647bc3..c4fcaaf95b6a1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q @@ -1,4 +1,4 @@ -dfs -put ../data/scripts/newline.py /newline.py; +dfs -put ../../data/scripts/newline.py /newline.py; add file hdfs:///newline.py; set hive.transform.escape.input=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q index 8d04d3e991c59..df199b0d7765b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q @@ -1,10 +1,12 @@ +DROP TABLE IF EXISTS repairtable; + CREATE TABLE repairtable(col STRING) PARTITIONED BY (p1 STRING, p2 STRING); MSCK TABLE repairtable; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/warehouse/repairtable/p1=a/p2=a; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/warehouse/repairtable/p1=b/p2=a; -dfs -touchz ../build/ql/test/data/warehouse/repairtable/p1=b/p2=a/datafile; +dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=a/p2=a; +dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=b/p2=a; +dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=b/p2=a/datafile; MSCK TABLE repairtable; @@ -12,4 +14,4 @@ MSCK REPAIR TABLE repairtable; MSCK TABLE repairtable; - +DROP TABLE repairtable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q new file mode 100644 index 0000000000000..a0514c86ff211 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q @@ -0,0 +1,11 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_root_dir_external_table; + +insert overwrite directory "hdfs:///tmp/test_root_dir_external_table" select key from src where (key < 20) order by key; + +dfs -cp /tmp/test_root_dir_external_table/000000_0 /000000_0; +dfs -rmr hdfs:///tmp/test_root_dir_external_table; + +create external table roottable (key string) row format delimited fields terminated by '\\t' stored as textfile location 'hdfs:///'; +select count(*) from roottable; + +dfs -rmr /000000_0; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q index ecd4d13d0e23b..b3c38bf577199 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q @@ -1,5 +1,5 @@ -dfs -mkdir file:///tmp/test; -dfs -mkdir hdfs:///tmp/test; +dfs ${system:test.dfs.mkdir} file:///tmp/test; +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test; create external table dynPart (key string) partitioned by (value string, value2 string) row format delimited fields terminated by '\\t' stored as textfile; insert overwrite local directory "/tmp/test" select key from src where (key = 10) order by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q index 4f65016f3081c..2dfb12951f9d3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q @@ -1,7 +1,9 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- EXCLUDE_OS_WINDOWS CREATE TABLE dest1(key INT, value STRING); -ADD FILE src/test/scripts/testgrep; +ADD FILE ../../ql/src/test/scripts/testgrep; FROM ( FROM src diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q new file mode 100644 index 0000000000000..0008ae51c4365 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q @@ -0,0 +1,16 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +-- INCLUDE_OS_WINDOWS + +CREATE TABLE dest1(key INT, value STRING); + +ADD FILE src/test/scripts/testgrep_win.bat; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'testgrep_win.bat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue; + +SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q new file mode 100644 index 0000000000000..25a1a81283221 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q @@ -0,0 +1,33 @@ +explain +select 'a', 100; +select 'a', 100; + +--evaluation +explain +select 1 + 1; +select 1 + 1; + +-- explode (not possible for lateral view) +explain +select explode(array('a', 'b')); +select explode(array('a', 'b')); + +set hive.fetch.task.conversion=more; + +explain +select 'a', 100; +select 'a', 100; + +explain +select 1 + 1; +select 1 + 1; + +explain +select explode(array('a', 'b')); +select explode(array('a', 'b')); + +-- subquery +explain +select 2 + 3,x from (select 1 + 2 x) X; +select 2 + 3,x from (select 1 + 2 x) X; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q index 2a287bd877759..accdb54744cc1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q @@ -31,8 +31,8 @@ WITH SERDEPROPERTIES ( ) 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; +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; SELECT * FROM serde_regex ORDER BY time; @@ -42,7 +42,7 @@ DROP TABLE serde_regex; EXPLAIN CREATE TABLE serde_regex1( - key decimal, + key decimal(38,18), value int) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' WITH SERDEPROPERTIES ( @@ -51,7 +51,7 @@ WITH SERDEPROPERTIES ( STORED AS TEXTFILE; CREATE TABLE serde_regex1( - key decimal, + key decimal(38,18), value int) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' WITH SERDEPROPERTIES ( @@ -59,7 +59,7 @@ WITH SERDEPROPERTIES ( ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH "../data/files/kv7.txt" INTO TABLE serde_regex1; +LOAD DATA LOCAL INPATH "../../data/files/kv7.txt" INTO TABLE serde_regex1; SELECT key, value FROM serde_regex1 ORDER BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q index 7e3d1f4d8aa41..d10239c31af62 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q @@ -24,7 +24,7 @@ set b=a; set c=${hiveconf:${hiveconf:b}}; set c; -set jar=${system:build.ivy.lib.dir}/default/derby-${system:derby.version}.jar; +set jar=${system:maven.local.repository}/org/apache/derby/derby/${system:derby.version}/derby-${system:derby.version}.jar; add file ${hiveconf:jar}; delete file ${hiveconf:jar}; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q index 5fcdf97e2db47..9758c16caa5ad 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; DROP TABLE show_idx_empty; DROP TABLE show_idx_full; @@ -24,4 +25,4 @@ SHOW INDEXES ON show_idx_empty; DROP INDEX idx_1 on show_idx_full; DROP INDEX idx_2 on show_idx_full; DROP TABLE show_idx_empty; -DROP TABLE show_idx_full; \ No newline at end of file +DROP TABLE show_idx_full; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q index ab588937e179b..bb43c5e1387a9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; DROP TABLE show_idx_t1; CREATE TABLE show_idx_t1(KEY STRING, VALUE STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q index 7fa7b828bd72e..1fc1d8e1f2a8b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q @@ -1,4 +1,5 @@ SHOW PARTITIONS srcpart; +SHOW PARTITIONS default.srcpart; SHOW PARTITIONS srcpart PARTITION(hr='11'); SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08'); -SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08', hr='12'); \ No newline at end of file +SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q new file mode 100644 index 0000000000000..d8ce96a37d7a8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q @@ -0,0 +1,4 @@ +create role role1; +create role role2; + +show roles; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q index 9184d6da897c0..55fb7b67ffc83 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q @@ -1,3 +1,4 @@ +set hive.support.quoted.identifiers=none; EXPLAIN SHOW TABLE EXTENDED IN default LIKE `src`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q index ad917beeef9ce..47535eab638d9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q @@ -13,10 +13,10 @@ CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q new file mode 100644 index 0000000000000..b8ca592ab70a1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q @@ -0,0 +1,9 @@ +set hive.auto.convert.join=false; +set hive.optimize.skewjoin=true; + +explain +create table noskew as select a.* from src a join src b on a.key=b.key order by a.key limit 30; + +create table noskew as select a.* from src a join src b on a.key=b.key order by a.key limit 30; + +select * from noskew; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q index 03eab4cd6d54e..fc07742cd7422 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q @@ -20,12 +20,12 @@ set mapred.input.dir.recursive=true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- a simple join query with skew on both the tables on the join key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q index 9cb919531f7c5..50cfc61962af9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q @@ -12,16 +12,16 @@ set mapred.input.dir.recursive=true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This is to test the union->selectstar->filesink and skewjoin optimization -- Union of 3 map-reduce subqueries is performed for the skew join diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q index af446bb65c08e..504ba8be2a29e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- a simple join query with skew on both the tables on the join key -- adding a order by at the end to make the results deterministic diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q index 199f3201afb77..f35af901704ef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q @@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; drop table array_valued_T1; create table array_valued_T1 (key string, value array) SKEWED BY (key) ON ((8)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q index ef61fb22f17a1..9e00bdcd76080 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q @@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- This test is to verify the skew join compile optimization when the join is followed -- by a union. Both sides of a union consist of a join, which should have used diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q index b5d9d9bc46874..171995069b77b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12), (8, 18)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key, val) ON ((3, 13), (8, 18)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- Both the join tables are skewed by 2 keys, and one of the skewed values -- is common to both the tables. The join key matches the skewed key set. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q index 0634c4f4ff6db..5ef217c90064a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q @@ -4,16 +4,16 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) SKEWED BY (val) ON ((12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This test is for skewed join compile time optimization for more than 2 tables. -- The join key for table 3 is different from the join key used for joining diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q index 0f031dd4fc0ae..df1a26bcc7d9d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q @@ -5,16 +5,16 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) SKEWED BY (val) ON ((12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This test is for skewed join compile time optimization for more than 2 tables. -- The join key for table 3 is different from the join key used for joining diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q index d5474a455e9dd..1db5472396db1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q @@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE tmpT1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE tmpT1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE tmpT1; -- testing skew on other data types - int CREATE TABLE T1(key INT, val STRING) SKEWED BY (key) ON ((2)); @@ -12,7 +12,7 @@ INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1; CREATE TABLE tmpT2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE tmpT2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE tmpT2; CREATE TABLE T2(key INT, val STRING) SKEWED BY (key) ON ((3)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q index 46b4f6d6e600d..915de612ded53 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- One of the tables is skewed by 2 columns, and the other table is -- skewed by one column. Ths join is performed on the both the columns diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q index 0592ca8c3e498..2ee79cc758531 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- One of the tables is skewed by 2 columns, and the other table is -- skewed by one column. Ths join is performed on the first skewed column @@ -31,12 +31,12 @@ DROP TABLE T2; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- One of the tables is skewed by 2 columns, and the other table is -- skewed by one column. Ths join is performed on the both the columns diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q index 433fea336dfd4..9d06cc030699a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q @@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE tmpT1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE tmpT1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE tmpT1; -- testing skew on other data types - int CREATE TABLE T1(key INT, val STRING) SKEWED BY (key) ON ((2)); @@ -16,7 +16,7 @@ INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- Once HIVE-3445 is fixed, the compile time skew join optimization would be -- applicable here. Till the above jira is fixed, it would be performed as a diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q index 0b11ebe4cb696..075645f89d452 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q @@ -6,11 +6,11 @@ CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) INTO 4 BUCKETS SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- add a test where the skewed key is also the bucketized key -- it should not matter, and the compile time skewed join diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q index 34fcdbfac4cb7..f7acaad18e1ea 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2), (7)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- a simple query with skew on both the tables on the join key -- multiple skew values are present for the skewed keys diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q index f217052881e24..9b908ce21b1d2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q @@ -6,11 +6,11 @@ CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- add a test where the skewed key is also the bucketized/sorted key -- it should not matter, and the compile time skewed join diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q index f6002ad49802a..22ea4f06218ac 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- a simple query with skew on both the tables. One of the skewed -- value is common to both the tables. The skewed value should not be diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q index ca83c446085fa..8496b1aa79c0a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q @@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- only of the tables of the join (the left table of the join) is skewed -- the skewed filter would still be applied to both the tables diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q index 3d7884c5e3dcb..152de5bde72c5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- One of the tables is skewed by 2 columns, and the other table is -- skewed by one column. Ths join is performed on the first skewed column diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q index 36cf8ceeaebb4..2e261bde66bbf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12), (8, 18)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key, val) ON ((3, 13), (8, 18)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- Both the join tables are skewed by 2 keys, and one of the skewed values -- is common to both the tables. The join key is a subset of the skewed key set: diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q index cf84f67b6a0fc..e4d9605f6f7af 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q @@ -5,16 +5,16 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This test is for validating skewed join compile time optimization for more than -- 2 tables. The join key is the same, and so a 3-way join would be performed. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q index d0ac845f86581..85746d9611dab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q @@ -4,16 +4,16 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This test is for validating skewed join compile time optimization for more than -- 2 tables. The join key is the same, and so a 3-way join would be performed. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q index 04834033a11e6..889ab6c3f5534 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q @@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- no skew join compile time optimization would be performed if one of the -- join sources is a sub-query consisting of a union all diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q index 359513e424dba..9dee4110f5991 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q index a79ebf62d0693..1fbe2090eaf2a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q @@ -6,11 +6,11 @@ alter table tmp_smb_bucket_10 add partition (ds = '2'); -- add dummy files to make sure that the number of files in each partition is same as number of buckets -load data local inpath '../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1'); -load data local inpath '../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1'); +load data local inpath '../../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1'); +load data local inpath '../../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1'); -load data local inpath '../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2'); -load data local inpath '../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2'); +load data local inpath '../../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2'); +load data local inpath '../../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2'); set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q index 9d86314879d68..e2b24333ad416 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q index 8b534e85aee1d..e43174bc0768b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q @@ -10,9 +10,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; explain select * from (select a.key from smb_bucket_1 a join smb_bucket_2 b on (a.key = b.key) where a.key = 5) t1 left outer join (select c.key from smb_bucket_2 c join smb_bucket_3 d on (c.key = d.key) where c.key=5) t2 on (t1.key=t2.key) where t2.key=5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q index 73b21fae250e0..b379706cc8ac5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q index 83143b170ed5c..2b3f67ea4eea1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q index 61ec084f64ffa..406604e621ad0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q index 1488b1f949527..ca1c7491b7298 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q @@ -15,8 +15,8 @@ create table smb_join_results(k1 int, v1 string, k2 int, v2 string); create table smb_join_results_empty_bigtable(k1 int, v1 string, k2 int, v2 string); create table normal_join_results(k1 int, v1 string, k2 int, v2 string); -load data local inpath '../data/files/empty1.txt' into table smb_bucket4_1; -load data local inpath '../data/files/empty2.txt' into table smb_bucket4_1; +load data local inpath '../../data/files/empty1.txt' into table smb_bucket4_1; +load data local inpath '../../data/files/empty2.txt' into table smb_bucket4_1; insert overwrite table smb_bucket4_2 select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q index 6f282ed441bfa..f296057d43e38 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q @@ -5,7 +5,7 @@ set hive.exec.reducers.max = 1; create table smb_bucket_input (key int, value string) stored as rcfile; -load data local inpath '../data/files/smb_bucket_input.rc' into table smb_bucket_input; +load data local inpath '../../data/files/smb_bucket_input.rc' into table smb_bucket_input; CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q index 6fe3d211a00b0..76ca152ef55f4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q @@ -1 +1 @@ -source ../data/files/source.txt; +source ../../data/files/source.txt; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q new file mode 100644 index 0000000000000..f5d7ff8fdd9bf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q @@ -0,0 +1,8 @@ +DROP TABLE tmp_jo_tab_test; +CREATE table tmp_jo_tab_test (message_line STRING) +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/input.txt' +OVERWRITE INTO TABLE tmp_jo_tab_test; + +select size(split(message_line, '\t')) from tmp_jo_tab_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q index 0b783de153b29..359d27b31523a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q @@ -26,5 +26,5 @@ DESCRIBE FORMATTED tmptable; -- Load a file into a existing table -- Some stats (numFiles, totalSize) should be updated correctly -- Some other stats (numRows, rawDataSize) should be cleared -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE tmptable; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE tmptable; DESCRIBE FORMATTED tmptable; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q index 6618c913ea700..d037c003b7582 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q @@ -2,25 +2,25 @@ set datanucleus.cache.collections=false; set hive.stats.autogather=true; CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; explain -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); desc formatted 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/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); desc formatted 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'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); desc formatted 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'); +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'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q index 425de64c26e83..e773cd749403f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q @@ -13,7 +13,7 @@ insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select -- Some other stats (numRows, rawDataSize) should be cleared desc formatted stats_part partition (ds='2010-04-08', hr='13'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE stats_part partition (ds='2010-04-08', hr='13'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE stats_part partition (ds='2010-04-08', hr='13'); desc formatted stats_part partition (ds='2010-04-08', hr='13'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q index da4af9655d16a..51514bd7738ff 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q @@ -1,7 +1,7 @@ set datanucleus.cache.collections=false; set hive.stats.autogather=true; set hive.stats.reliable=true; -set hive.stats.dbclass=dummy; +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.KeyVerifyingStatsAggregator; @@ -56,7 +56,7 @@ insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select desc formatted stats_part partition (ds='2010-04-08', hr = '13'); -set hive.stats.dbclass=dummy; +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.KeyVerifyingStatsAggregator; set hive.stats.key.prefix.max.length=0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q index 5962348d9c317..fd7e0eaca8c39 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q @@ -5,9 +5,9 @@ drop table hive_test_dst; create table hive_test_src ( col1 string ) stored as textfile ; explain extended -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; desc formatted hive_test_src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q index 62580042d4ded..80a67f405cb0b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q @@ -25,8 +25,8 @@ insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, v show partitions nzhang_part1; show partitions nzhang_part2; -select * from nzhang_part1 where ds is not null and hr is not null; -select * from nzhang_part2 where ds is not null and hr is not null; +select * from nzhang_part1 where ds is not null and hr is not null order by ds, hr, key; +select * from nzhang_part2 where ds is not null and hr is not null order by ds, hr, key; describe formatted nzhang_part1 partition(ds='2008-04-08',hr=11); describe formatted nzhang_part1 partition(ds='2008-04-08',hr=12); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q index 4e7d3dc547a30..5e6b0aaa1253b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q @@ -1,12 +1,12 @@ -- In this test, there is a dummy stats aggregator which throws an error when various --- methods are called (as indicated by the parameter hive.test.dummystats.agregator) +-- methods are called (as indicated by the parameter hive.test.dummystats.aggregator) -- Since stats need not be reliable (by setting hive.stats.reliable to false), the -- insert statements succeed. The insert statement succeeds even if the stats aggregator -- is set to null, since stats need not be reliable. create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +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.stats.reliable=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q new file mode 100644 index 0000000000000..3c1f132a68f2f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q @@ -0,0 +1,16 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=false; + +-- by analyze +create table dummy1 as select * from src; + +analyze table dummy1 compute statistics; +desc formatted dummy1; + +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; + +-- by autogather +create table dummy2 as select * from src; + +desc formatted dummy2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q new file mode 100644 index 0000000000000..e1274c0cb5197 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q @@ -0,0 +1,45 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +-- partitioned table analyze + +create table dummy (key string, value string) partitioned by (ds string, hr string); + +load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='12'); +load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='11'); + +analyze table dummy partition (ds,hr) compute statistics; +describe formatted dummy partition (ds='2008', hr='11'); +describe formatted dummy partition (ds='2008', hr='12'); + +drop table dummy; + +-- static partitioned table on insert + +create table dummy (key string, value string) partitioned by (ds string, hr string); + +insert overwrite table dummy partition (ds='10',hr='11') select * from src; +insert overwrite table dummy partition (ds='10',hr='12') select * from src; + +describe formatted dummy partition (ds='10', hr='11'); +describe formatted dummy partition (ds='10', hr='12'); + +drop table dummy; + +-- dynamic partitioned table on insert + +create table dummy (key int) partitioned by (hr int); + +CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; +LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl; + +insert overwrite table dummy partition (hr) select * from tbl; + +describe formatted dummy partition (hr=1997); +describe formatted dummy partition (hr=1994); +describe formatted dummy partition (hr=1998); +describe formatted dummy partition (hr=1996); + +drop table tbl; +drop table dummy; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q new file mode 100644 index 0000000000000..a7fce6e3e503f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q @@ -0,0 +1,15 @@ +set hive.stats.autogather=true; + +CREATE TABLE stats_invalid (key string, value string); + +insert overwrite table stats_invalid +select * from src; + +analyze table stats_invalid compute statistics for columns key,value; + +desc formatted stats_invalid; +alter table stats_invalid add columns (new_col string); + +desc formatted stats_invalid; +drop table stats_invalid; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q new file mode 100644 index 0000000000000..5982643741548 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q @@ -0,0 +1,45 @@ + +set hive.mapred.supports.subdirectories=true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +drop table stats_list_bucket; +drop table stats_list_bucket_1; + +create table stats_list_bucket ( + c1 string, + c2 string +) partitioned by (ds string, hr string) +skewed by (c1, c2) on (('466','val_466'),('287','val_287'),('82','val_82')) +stored as directories +stored as rcfile; + +set hive.stats.key.prefix.max.length=1; + +-- Make sure we use hashed IDs during stats publishing. +-- Try partitioned table with list bucketing. +-- The stats should show 500 rows loaded, as many rows as the src table has. + +insert overwrite table stats_list_bucket partition (ds = '2008-04-08', hr = '11') + select key, value from src; + +desc formatted stats_list_bucket partition (ds = '2008-04-08', hr = '11'); + +-- Also try non-partitioned table with list bucketing. +-- Stats should show the same number of rows. + +create table stats_list_bucket_1 ( + c1 string, + c2 string +) +skewed by (c1, c2) on (('466','val_466'),('287','val_287'),('82','val_82')) +stored as directories +stored as rcfile; + +insert overwrite table stats_list_bucket_1 + select key, value from src; + +desc formatted stats_list_bucket_1; + +drop table stats_list_bucket; +drop table stats_list_bucket_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q index c934fb2051258..b106b30476c00 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q @@ -1,12 +1,12 @@ -- test analyze table compute statistiscs [noscan] on external table -- 1 test table -CREATE EXTERNAL TABLE anaylyze_external (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test'; +CREATE EXTERNAL TABLE anaylyze_external (a INT) LOCATION '${system:hive.root}/data/files/ext_test'; SELECT * FROM anaylyze_external; -analyze table anaylyze_external compute statistics; -describe formatted anaylyze_external; analyze table anaylyze_external compute statistics noscan; describe formatted anaylyze_external; +analyze table anaylyze_external compute statistics; +describe formatted anaylyze_external; drop table anaylyze_external; -- 2 test partition @@ -21,10 +21,10 @@ CREATE EXTERNAL TABLE anaylyze_external (key string, val string) partitioned by ALTER TABLE anaylyze_external ADD PARTITION (insertdate='2008-01-01') location 'pfile://${system:test.tmp.dir}/texternal/2008-01-01'; select count(*) from anaylyze_external where insertdate='2008-01-01'; -- analyze -analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics; -describe formatted anaylyze_external PARTITION (insertdate='2008-01-01'); analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics noscan; describe formatted anaylyze_external PARTITION (insertdate='2008-01-01'); +analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics; +describe formatted anaylyze_external PARTITION (insertdate='2008-01-01'); dfs -rmr ${system:test.tmp.dir}/texternal; drop table anaylyze_external; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q new file mode 100644 index 0000000000000..b47bc48958c8d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q @@ -0,0 +1,41 @@ +set hive.stats.dbclass=fs; +set hive.compute.query.using.stats=true; +set hive.stats.autogather=true; +CREATE TABLE temps_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE; + +CREATE TABLE stats_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE; + +CREATE TABLE stats_null_part(a double, b int, c STRING, d smallint) partitioned by (dt string) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE temps_null; + +insert overwrite table stats_null select * from temps_null; +insert into table stats_null_part partition(dt='2010') select * from temps_null where d <=5; + +insert into table stats_null_part partition(dt='2011') select * from temps_null where d > 5; +explain +select count(*), count(a), count(b), count(c), count(d) from stats_null; +explain +select count(*), count(a), count(b), count(c), count(d) from stats_null_part; + + +analyze table stats_null compute statistics for columns a,b,c,d; +analyze table stats_null_part partition(dt='2010') compute statistics for columns a,b,c,d; +analyze table stats_null_part partition(dt='2011') compute statistics for columns a,b,c,d; + +describe formatted stats_null_part partition (dt='2010'); +describe formatted stats_null_part partition (dt='2011'); + +explain +select count(*), count(a), count(b), count(c), count(d) from stats_null; +explain +select count(*), count(a), count(b), count(c), count(d) from stats_null_part; + + +select count(*), count(a), count(b), count(c), count(d) from stats_null; +select count(*), count(a), count(b), count(c), count(d) from stats_null_part; +drop table stats_null; +drop table stats_null_part; +drop table temps_null; +set hive.compute.query.using.stats=false; +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q index 6d383f213d1be..513b8e75a0c5e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q @@ -6,7 +6,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +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.stats.reliable=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q new file mode 100644 index 0000000000000..82a2295ac27b4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q @@ -0,0 +1,63 @@ +set hive.stats.dbclass=fs; + +-- stats computation on partitioned table with analyze command + +create table t1 (key string, value string) partitioned by (ds string); +load data local inpath '../../data/files/kv1.txt' into table t1 partition (ds = '2010'); +load data local inpath '../../data/files/kv1.txt' into table t1 partition (ds = '2011'); + +analyze table t1 partition (ds) compute statistics; + +describe formatted t1 partition (ds='2010'); +describe formatted t1 partition (ds='2011'); + +drop table t1; + +-- stats computation on partitioned table with autogather on insert query + +create table t1 (key string, value string) partitioned by (ds string); + +insert into table t1 partition (ds='2010') select * from src; +insert into table t1 partition (ds='2011') select * from src; + +describe formatted t1 partition (ds='2010'); +describe formatted t1 partition (ds='2011'); + +drop table t1; + +-- analyze stmt on unpartitioned table + +create table t1 (key string, value string); +load data local inpath '../../data/files/kv1.txt' into table t1; + +analyze table t1 compute statistics; + +describe formatted t1 ; + +drop table t1; + +-- stats computation on unpartitioned table with autogather on insert query + +create table t1 (key string, value string); + +insert into table t1 select * from src; + +describe formatted t1 ; + +drop table t1; + +-- stats computation on partitioned table with autogather on insert query with dynamic partitioning + + +create table t1 (key string, value string) partitioned by (ds string, hr string); + +set hive.exec.dynamic.partition.mode=nonstrict; +insert into table t1 partition (ds,hr) select * from srcpart; + +describe formatted t1 partition (ds='2008-04-08',hr='11'); +describe formatted t1 partition (ds='2008-04-09',hr='12'); + +drop table t1; +set hive.exec.dynamic.partition.mode=strict; + +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q index c3b206bba6325..ae83407f84333 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + desc function str_to_map; desc function extended str_to_map; @@ -19,7 +21,7 @@ limit 3; drop table tbl_s2m; -create table tbl_s2m as select 'ABC=CC_333=444' as t from src limit 3; +create table tbl_s2m as select 'ABC=CC_333=444' as t from src tablesample (3 rows); select str_to_map(t,'_','=')['333'] from tbl_s2m; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q index 3fb1558a90f0b..14fa321c11c2f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q @@ -2,12 +2,12 @@ EXPLAIN FROM ( FROM src select src.* WHERE src.key < 100 ) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*; +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; FROM ( FROM src select src.* WHERE src.key < 100 ) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*; +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; -dfs -cat ../build/ql/test/data/warehouse/union.out/*; +dfs -cat ${system:test.warehouse.dir}/union.out/*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q new file mode 100644 index 0000000000000..1d539825424f0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q @@ -0,0 +1,5 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask.size=10000000; +explain select src.key from src where src.key in ( select distinct key from src); + +set hive.auto.convert.join=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q new file mode 100644 index 0000000000000..ffc33dc2cf852 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q @@ -0,0 +1,16 @@ +EXPLAIN +FROM ( + FROM src select src.* WHERE src.key < 100 +) as unioninput +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; + +EXPLAIN +SELECT * FROM +( SELECT * FROM + ( SELECT * FROM src as s ) as src1 +) as src2; + +SELECT * FROM +( SELECT * FROM + ( SELECT * FROM src as s ) as src1 +) as src2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q new file mode 100644 index 0000000000000..f812e36070023 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q @@ -0,0 +1,45 @@ + + +-- no agg, corr +explain +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9' + ) +; + +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9' + ) +; + +-- view test +create view cv1 as +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +; + +select * from cv1 +; + +-- sq in from +select * +from (select * + from src b + where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') + ) a +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q new file mode 100644 index 0000000000000..690aa10527a89 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q @@ -0,0 +1,60 @@ + + +-- no agg, corr +explain +select b.key, count(*) +from src b +group by b.key +having exists + (select a.key + from src a + where a.key = b.key and a.value > 'val_9' + ) +; + +select b.key, count(*) +from src b +group by b.key +having exists + (select a.key + from src a + where a.key = b.key and a.value > 'val_9' + ) +; + +-- view test +create view cv1 as +select b.key, count(*) as c +from src b +group by b.key +having exists + (select a.key + from src a + where a.key = b.key and a.value > 'val_9' + ) +; + +select * from cv1; + +-- sq in from +select * +from (select b.key, count(*) + from src b + group by b.key + having exists + (select a.key + from src a + where a.key = b.key and a.value > 'val_9' + ) +) a +; + +-- join on agg +select b.key, min(b.value) +from src b +group by b.key +having exists ( select a.key + from src a + where a.value > 'val_9' and a.value = min(b.value) + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q new file mode 100644 index 0000000000000..69f40f9b8ca9a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q @@ -0,0 +1,163 @@ +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; + +DROP TABLE lineitem; +CREATE TABLE lineitem (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|'; + +LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem; + +-- non agg, non corr +explain + select * +from src +where src.key in (select key from src s1 where s1.key > '9') +; + +select * +from src +where src.key in (select key from src s1 where s1.key > '9') +order by key +; + +-- non agg, corr +explain +select * +from src b +where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +; + +select * +from src b +where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +order by b.key +; + +-- agg, non corr +explain +select p_name, p_size +from +part where part.p_size in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +; +select p_name, p_size +from +part where part.p_size in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +order by p_name +; + +-- agg, corr +explain +select p_mfgr, p_name, p_size +from part b where b.p_size in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +; + +select p_mfgr, p_name, p_size +from part b where b.p_size in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +order by p_mfgr, p_name, p_size +; + +-- distinct, corr +explain +select * +from src b +where b.key in + (select distinct a.key + from src a + where b.value = a.value and a.key > '9' + ) +; + +select * +from src b +where b.key in + (select distinct a.key + from src a + where b.value = a.value and a.key > '9' + ) +order by b.key +; + +-- non agg, non corr, windowing +select p_mfgr, p_name, p_size +from part +where part.p_size in + (select first_value(p_size) over(partition by p_mfgr order by p_size) from part) +order by p_mfgr, p_name, p_size +; + +-- non agg, non corr, with join in Parent Query +explain +select p.p_partkey, li.l_suppkey +from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey +where li.l_linenumber = 1 and + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR') +; + +select p.p_partkey, li.l_suppkey +from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey +where li.l_linenumber = 1 and + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR') +order by p.p_partkey, li.l_suppkey +; + +-- non agg, corr, with join in Parent Query +select p.p_partkey, li.l_suppkey +from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey +where li.l_linenumber = 1 and + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +order by p.p_partkey, li.l_suppkey +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q new file mode 100644 index 0000000000000..84045568f4501 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q @@ -0,0 +1,104 @@ + +-- 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; + +-- non agg, non corr +explain + select key, count(*) +from src +group by key +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +; + + +select s1.key, count(*) from src s1 where s1.key > '9' group by s1.key order by s1.key; + +select key, count(*) +from src +group by key +having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key ) +order by key +; + +-- non agg, corr +explain + select key, value, count(*) +from src b +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' and s1.value = b.value group by s1.key ) +; + +-- agg, non corr +explain +select p_mfgr, avg(p_size) +from part b +group by b.p_mfgr +having b.p_mfgr in + (select p_mfgr + from part + group by p_mfgr + having max(p_size) - min(p_size) < 20 + ) +; + +-- join on agg +select b.key, min(b.value) +from src b +group by b.key +having b.key in ( select a.key + from src a + where a.value > 'val_9' and a.value = min(b.value) + ) +order by b.key +; + +-- where and having +-- Plan is: +-- Stage 1: b semijoin sq1:src (subquery in where) +-- Stage 2: group by Stage 1 o/p +-- Stage 5: group by on sq2:src (subquery in having) +-- Stage 6: Stage 2 o/p semijoin Stage 5 +explain +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +; + +set hive.auto.convert.join=true; +-- Plan is: +-- Stage 5: group by on sq2:src (subquery in having) +-- Stage 10: hashtable for sq1:src (subquery in where) +-- Stage 2: b map-side semijoin Stage 10 o/p +-- Stage 3: Stage 2 semijoin Stage 5 +-- Stage 9: construct hastable for Stage 5 o/p +-- Stage 6: Stage 2 map-side semijoin Stage 9 +explain +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +; + +-- non agg, non corr, windowing +explain +select p_mfgr, p_name, avg(p_size) +from part +group by p_mfgr, p_name +having p_name in + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q new file mode 100644 index 0000000000000..ed36d9ef6e961 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q @@ -0,0 +1,82 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; + +CREATE TABLE src_4( + key STRING, + value STRING +) +; + +CREATE TABLE src_5( + key STRING, + value STRING +) +; + +explain +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +; + +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +; + +select * from src_4 +; +select * from src_5 +; +set hive.auto.convert.join=true; + +explain +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +; + +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +; + +select * from src_4 +; +select * from src_5 +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q new file mode 100644 index 0000000000000..43a801fa9683e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q @@ -0,0 +1,41 @@ + + +-- no agg, corr +explain +select * +from src b +where not exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_2' + ) +; + +select * +from src b +where not exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_2' + ) +; + +-- distinct, corr +explain +select * +from src b +where not exists + (select distinct a.key + from src a + where b.value = a.value and a.value > 'val_2' + ) +; + +select * +from src b +where not exists + (select a.key + from src a + where b.value = a.value and a.value > 'val_2' + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q new file mode 100644 index 0000000000000..7205d17bc4861 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q @@ -0,0 +1,46 @@ + + +-- no agg, corr +explain +select * +from src b +group by key, value +having not exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_12' + ) +; + +select * +from src b +group by key, value +having not exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_12' + ) +; + + +-- distinct, corr +explain +select * +from src b +group by key, value +having not exists + (select distinct a.key + from src a + where b.value = a.value and a.value > 'val_12' + ) +; + +select * +from src b +group by key, value +having not exists + (select distinct a.key + from src a + where b.value = a.value and a.value > 'val_12' + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q new file mode 100644 index 0000000000000..d5f60860313da --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q @@ -0,0 +1,143 @@ +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; + +DROP TABLE lineitem; +CREATE TABLE lineitem (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|'; + +LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem; + +-- non agg, non corr +explain +select * +from src +where src.key not in + ( select key from src s1 + where s1.key > '2' + ) +; + +select * +from src +where src.key not in ( select key from src s1 where s1.key > '2') +order by key +; + +-- non agg, corr +explain +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +; + +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +order by p_mfgr, b.p_name +; + +-- agg, non corr +explain +select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +; +select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +order by p_name, p_size +; + +-- agg, corr +explain +select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +; + +select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +order by p_mfgr, p_size +; + +-- non agg, non corr, Group By in Parent Query +select li.l_partkey, count(*) +from lineitem li +where li.l_linenumber = 1 and + li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') +group by li.l_partkey +; + +-- alternate not in syntax +select * +from src +where not src.key in ( select key from src s1 where s1.key > '2') +order by key +; + +-- null check +create view T1_v as +select key from src where key <'11'; + +create view T2_v as +select case when key > '104' then null else key end as key from T1_v; + +explain +select * +from T1_v where T1_v.key not in (select T2_v.key from T2_v); + +select * +from T1_v where T1_v.key not in (select T2_v.key from T2_v); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q new file mode 100644 index 0000000000000..a586f02272a73 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q @@ -0,0 +1,74 @@ +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; + + +-- non agg, non corr +explain +select key, count(*) +from src +group by key +having key not in + ( select key from src s1 + where s1.key > '12' + ) +; + +-- non agg, corr +explain +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a + where min(p_retailprice) = l and r - l > 600 + ) +; + +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a + where min(p_retailprice) = l and r - l > 600 + ) +; + +-- agg, non corr +explain +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from part a + group by p_mfgr + having max(p_retailprice) - min(p_retailprice) > 600 + ) +; + +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from part a + group by p_mfgr + having max(p_retailprice) - min(p_retailprice) > 600 + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q new file mode 100644 index 0000000000000..749435c005134 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q @@ -0,0 +1,83 @@ +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 +); + +create table src11 (key1 string, value1 string); + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +-- non agg, corr +explain select * from src11 where src11.key1 in (select key from src where src11.value1 = value and key > '9'); + +explain select * from src a where a.key in (select key from src where a.value = value and key > '9'); + +-- agg, corr +explain +select p_mfgr, p_name, p_size +from part b where b.p_size in + (select min(p2_size) + from (select p2_mfgr, p2_size, rank() over(partition by p2_mfgr order by p2_size) as r from part2) a + where r <= 2 and b.p_mfgr = p2_mfgr + ) +; + + +explain +select p_mfgr, p_name, p_size +from part b where b.p_size in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = p_mfgr + ) +; + +-- distinct, corr +explain +select * +from src b +where b.key in + (select distinct key + from src + where b.value = value and key > '9' + ) +; + +-- non agg, corr, having +explain + select key, value, count(*) +from src b +group by key, value +having count(*) in (select count(*) from src where src.key > '9' and src.value = b.value group by key ) +; + +-- non agg, corr +explain +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = p_mfgr + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q new file mode 100644 index 0000000000000..9f6712fc181ff --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q @@ -0,0 +1,48 @@ + + +-- exists test +create view cv1 as +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +; + +select * +from cv1 where cv1.key in (select key from cv1 c where c.key > '95') order by key; +; + + +-- not in test +create view cv2 as +select * +from src b +where b.key not in + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_11' + ) +; + +select * +from cv2 where cv2.key in (select key from cv2 c where c.key < '11') order by key; +; + +-- in where + having +create view cv3 as +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +; + +select * from cv3 order by key; + + +-- join of subquery views +select * +from cv3 +where cv3.key in (select key from cv1) order by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q index bb9d6f34ed8ec..d633b97f4c931 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q @@ -1,12 +1,12 @@ - +DROP TABLE IF EXISTS symlink_text_input_format; EXPLAIN CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; -dfs -cp ../data/files/symlink1.txt ../build/ql/test/data/warehouse/symlink_text_input_format/symlink1.txt; -dfs -cp ../data/files/symlink2.txt ../build/ql/test/data/warehouse/symlink_text_input_format/symlink2.txt; +dfs -cp ../../data/files/symlink1.txt ${system:test.warehouse.dir}/symlink_text_input_format/symlink1.txt; +dfs -cp ../../data/files/symlink2.txt ${system:test.warehouse.dir}/symlink_text_input_format/symlink2.txt; EXPLAIN SELECT * FROM symlink_text_input_format order by key, value; @@ -20,5 +20,4 @@ EXPLAIN SELECT count(1) FROM symlink_text_input_format; SELECT count(1) FROM symlink_text_input_format; - - +DROP TABLE symlink_text_input_format; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q index 8b1a390149865..23209d85e4f15 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q @@ -4,7 +4,7 @@ SET hive.stats.collect.tablekeys=true; -- This test is used for testing the TableAccessAnalyzer CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q index d2da5ac174a15..a4f0fdb1c1f26 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q @@ -1,5 +1,5 @@ create table if not exists test_boolean(dummy tinyint); -insert overwrite table test_boolean select 1 from src limit 1; +insert overwrite table test_boolean select 1 from src tablesample (1 rows); SELECT 1 FROM ( diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q new file mode 100644 index 0000000000000..87d251f40fd7a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q @@ -0,0 +1,40 @@ +set hive.exec.dynamic.partition.mode=nonstrict; + +-- CTAS +EXPLAIN CREATE TABLE tmp_src AS SELECT * FROM (SELECT value, count(value) AS cnt FROM src GROUP BY value) f1 ORDER BY cnt; +CREATE TABLE tmp_src AS SELECT * FROM (SELECT value, count(value) AS cnt FROM src GROUP BY value) f1 ORDER BY cnt; + +SELECT * FROM tmp_src; + +-- dyn partitions +CREATE TABLE tmp_src_part (c string) PARTITIONED BY (d int); +EXPLAIN INSERT INTO TABLE tmp_src_part PARTITION (d) SELECT * FROM tmp_src; +INSERT INTO TABLE tmp_src_part PARTITION (d) SELECT * FROM tmp_src; + +SELECT * FROM tmp_src_part; + +-- multi insert +CREATE TABLE even (c int, d string); +CREATE TABLE odd (c int, d string); + +EXPLAIN +FROM src +INSERT INTO TABLE even SELECT key, value WHERE key % 2 = 0 +INSERT INTO TABLE odd SELECT key, value WHERE key % 2 = 1; + +FROM src +INSERT INTO TABLE even SELECT key, value WHERE key % 2 = 0 +INSERT INTO TABLE odd SELECT key, value WHERE key % 2 = 1; + +SELECT * FROM even; +SELECT * FROM odd; + +-- create empty table +CREATE TABLE empty STORED AS orc AS SELECT * FROM tmp_src_part WHERE d = -1000; +SELECT * FROM empty; + +-- drop the tables +DROP TABLE even; +DROP TABLE odd; +DROP TABLE tmp_src; +DROP TABLE tmp_src_part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q new file mode 100644 index 0000000000000..7f2e28fbafa8d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q @@ -0,0 +1,19 @@ +set hive.execution.engine=tez; +CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +CREATE TABLE t1 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE t1 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE t1 partition(ds='2008-04-08'); + +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; +set hive.stats.dbclass=fs; + +insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from t1; +describe formatted tab_part partition(ds='2008-04-08'); + +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q new file mode 100644 index 0000000000000..d7a652fb8c8bd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q @@ -0,0 +1,5 @@ +insert overwrite local directory '${system:test.tmp.dir}/tez_local_src_table_1' +select * from src order by key limit 10 ; +dfs -cat file:${system:test.tmp.dir}/tez_local_src_table_1/000000_0 ; + +dfs -rmr file:${system:test.tmp.dir}/tez_local_src_table_1/ ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q new file mode 100644 index 0000000000000..f309e3fe0eed6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q @@ -0,0 +1,12 @@ +explain +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key; + +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key; +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key)) x right outer join src c on (x.value = c.value) order by x.key; +select * from src1 a left outer join src b on (a.key = b.key) right outer join src c on (a.value = c.value) order by a.key; +select * from src1 a left outer join src b on (a.key = b.key) left outer join src c on (a.value = c.value) order by a.key; +select * from src1 a left outer join src b on (a.key = b.key) join src c on (a.key = c.key); +select * from src1 a join src b on (a.key = b.key) join src c on (a.key = c.key); + +select count(*) from src1 a join src b on (a.key = b.key) join src c on (a.key = c.key); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q new file mode 100644 index 0000000000000..9193843824f6d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q @@ -0,0 +1,5 @@ +explain +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key; + +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q new file mode 100644 index 0000000000000..2f1c73f8e528a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q @@ -0,0 +1,14 @@ +create table test (key int, value string) partitioned by (p int) stored as textfile; + +insert into table test partition (p=1) select * from src limit 10; + +alter table test set fileformat orc; + +insert into table test partition (p=2) select * from src limit 10; + +describe test; + +select * from test where p=1 and key > 0; +select * from test where p=2 and key > 0; +select * from test where key > 0; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q new file mode 100644 index 0000000000000..f80d94c4a15fd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q @@ -0,0 +1,94 @@ +set hive.auto.convert.join=true; + +explain +select s1.key as key, s1.value as value from src s1 join src s3 on s1.key=s3.key +UNION ALL +select s2.key as key, s2.value as value from src s2; + +create table ut as +select s1.key as key, s1.value as value from src s1 join src s3 on s1.key=s3.key +UNION ALL +select s2.key as key, s2.value as value from src s2; + +select * from ut order by key, value limit 20; +drop table ut; + +set hive.auto.convert.join=false; + +explain +with u as (select * from src union all select * from src) +select count(*) from (select u1.key as k1, u2.key as k2 from +u as u1 join u as u2 on (u1.key = u2.key)) a; + +create table ut as +with u as (select * from src union all select * from src) +select count(*) as cnt from (select u1.key as k1, u2.key as k2 from +u as u1 join u as u2 on (u1.key = u2.key)) a; + +select * from ut order by cnt limit 20; +drop table ut; + +set hive.auto.convert.join=true; + +explain select s1.key as skey, u1.key as ukey from +src s1 +join (select * from src union all select * from src) u1 on s1.key = u1.key; + +create table ut as +select s1.key as skey, u1.key as ukey from +src s1 +join (select * from src union all select * from src) u1 on s1.key = u1.key; + +select * from ut order by skey, ukey limit 20; +drop table ut; + +explain select s1.key as skey, u1.key as ukey, s8.key as lkey from +src s1 +join (select s2.key as key from src s2 join src s3 on s2.key = s3.key + union all select s4.key from src s4 join src s5 on s4.key = s5.key + union all select s6.key from src s6 join src s7 on s6.key = s7.key) u1 on (s1.key = u1.key) +join src s8 on (u1.key = s8.key) +order by lkey; + +create table ut as +select s1.key as skey, u1.key as ukey, s8.key as lkey from +src s1 +join (select s2.key as key from src s2 join src s3 on s2.key = s3.key + union all select s4.key from src s4 join src s5 on s4.key = s5.key + union all select s6.key from src s6 join src s7 on s6.key = s7.key) u1 on (s1.key = u1.key) +join src s8 on (u1.key = s8.key) +order by lkey; + +select * from ut order by skey, ukey, lkey limit 100; + +drop table ut; + +explain +select s2.key as key from src s2 join src s3 on s2.key = s3.key +union all select s4.key from src s4 join src s5 on s4.key = s5.key; + +create table ut as +select s2.key as key from src s2 join src s3 on s2.key = s3.key +union all select s4.key from src s4 join src s5 on s4.key = s5.key; + +select * from ut order by key limit 30; + +drop table ut; + +explain +select * from +(select * from src union all select * from src) u +left outer join src s on u.key = s.key; + +explain +select u.key as ukey, s.key as skey from +(select * from src union all select * from src) u +right outer join src s on u.key = s.key; + +create table ut as +select u.key as ukey, s.key as skey from +(select * from src union all select * from src) u +right outer join src s on u.key = s.key; + +select * from ut order by ukey, skey limit 20; +drop table ut; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q index f2c3b596af44a..ce79eefaae4d3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q @@ -1,10 +1,12 @@ +set hive.fetch.task.conversion=more; + drop table timestamp_1; create table timestamp_1 (t timestamp); alter table timestamp_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; -insert overwrite table timestamp_1 - select cast('2011-01-01 01:01:01' as timestamp) from src limit 1; +insert overwrite table timestamp_1 + select cast('2011-01-01 01:01:01' as timestamp) from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -15,7 +17,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01' from src limit 1; + select '2011-01-01 01:01:01' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -26,7 +28,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01.1' from src limit 1; + select '2011-01-01 01:01:01.1' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -37,7 +39,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01.0001' from src limit 1; + select '2011-01-01 01:01:01.0001' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -48,7 +50,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01.000100000' from src limit 1; + select '2011-01-01 01:01:01.000100000' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -59,7 +61,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01.001000011' from src limit 1; + select '2011-01-01 01:01:01.001000011' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q index b93208f48c453..351f5ca519499 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q @@ -1,10 +1,12 @@ +set hive.fetch.task.conversion=more; + drop table timestamp_2; create table timestamp_2 (t timestamp); alter table timestamp_2 set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; -insert overwrite table timestamp_2 - select cast('2011-01-01 01:01:01' as timestamp) from src limit 1; +insert overwrite table timestamp_2 + select cast('2011-01-01 01:01:01' as timestamp) from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -15,7 +17,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01' from src limit 1; + select '2011-01-01 01:01:01' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -26,7 +28,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01.1' from src limit 1; + select '2011-01-01 01:01:01.1' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -37,7 +39,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01.0001' from src limit 1; + select '2011-01-01 01:01:01.0001' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -48,7 +50,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01.000100000' from src limit 1; + select '2011-01-01 01:01:01.000100000' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -59,7 +61,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01.001000011' from src limit 1; + select '2011-01-01 01:01:01.001000011' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q index cda724f9e8f43..0e1a8d5526783 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q @@ -1,10 +1,12 @@ +set hive.fetch.task.conversion=more; + drop table timestamp_3; create table timestamp_3 (t timestamp); alter table timestamp_3 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; -insert overwrite table timestamp_3 - select cast(cast('1.3041352164485E9' as double) as timestamp) from src limit 1; +insert overwrite table timestamp_3 + select cast(cast('1.3041352164485E9' as double) as timestamp) from src tablesample (1 rows); select cast(t as boolean) from timestamp_3 limit 1; select cast(t as tinyint) from timestamp_3 limit 1; select cast(t as smallint) from timestamp_3 limit 1; @@ -14,4 +16,6 @@ select cast(t as float) from timestamp_3 limit 1; select cast(t as double) from timestamp_3 limit 1; select cast(t as string) from timestamp_3 limit 1; +select t, sum(t), count(*), sum(t)/count(*), avg(t) from timestamp_3 group by t; + drop table timestamp_3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q index f64ae48b85db7..30fee3cbf6013 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q @@ -1,5 +1,6 @@ +set hive.fetch.task.conversion=more; -select cast('2011-05-06 07:08:09' as timestamp) > +select cast('2011-05-06 07:08:09' as timestamp) > cast('2011-05-06 07:08:09' as timestamp) from src limit 1; select cast('2011-05-06 07:08:09' as timestamp) < diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q index 7a1005295eb28..e9a0cfae7cc9a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q @@ -1,6 +1,6 @@ drop table timestamp_lazy; create table timestamp_lazy (t timestamp, key string, value string); -insert overwrite table timestamp_lazy select cast('2011-01-01 01:01:01' as timestamp), key, value from src limit 5; +insert overwrite table timestamp_lazy select cast('2011-01-01 01:01:01' as timestamp), key, value from src tablesample (5 rows); select t,key,value from timestamp_lazy ORDER BY key ASC, value ASC; select t,key,value from timestamp_lazy distribute by t sort by key ASC, value ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q index efd5bc4b78bf4..36f35413e9b48 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS timestamp_null; CREATE TABLE timestamp_null (t1 TIMESTAMP); -LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE timestamp_null; +LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE timestamp_null; SELECT * FROM timestamp_null LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q index 2620acefee7f7..ade9fb408c092 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q @@ -1,13 +1,15 @@ +set hive.fetch.task.conversion=more; + drop table timestamp_udf; drop table timestamp_udf_string; create table timestamp_udf (t timestamp); create table timestamp_udf_string (t string); -from src +from (select * from src tablesample (1 rows)) s insert overwrite table timestamp_udf - select '2011-05-06 07:08:09.1234567' limit 1 + select '2011-05-06 07:08:09.1234567' insert overwrite table timestamp_udf_string - select '2011-05-06 07:08:09.1234567' limit 1; + select '2011-05-06 07:08:09.1234567'; -- Test UDFs with Timestamp input select unix_timestamp(t), year(t), month(t), day(t), dayofmonth(t), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q index 962077c2ca565..3bed2b6727e7b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q @@ -12,7 +12,7 @@ SELECT transform(*) USING 'cat' AS (col array) FROM transform1_t1; create table transform1_t2(col array); insert overwrite table transform1_t2 -select array(1,2,3) from src limit 1; +select array(1,2,3) from src tablesample (1 rows); EXPLAIN SELECT transform('0\0021\0022') USING 'cat' AS (col array) FROM transform1_t2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q index d756b47e464d0..0bfb23ead6869 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q @@ -5,7 +5,7 @@ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' STORED A set hive.stats.autogather=true; -INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows); DESC FORMATTED test_tab; @@ -20,7 +20,7 @@ DESC FORMATTED test_tab; SELECT * FROM test_tab ORDER BY value; -- Truncate multiple columns -INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows); TRUNCATE TABLE test_tab COLUMNS (key, value); @@ -40,7 +40,7 @@ SELECT * FROM test_tab ORDER BY value; -- Test truncating with a binary serde ALTER TABLE test_tab SET SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; -INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows); DESC FORMATTED test_tab; @@ -65,7 +65,7 @@ SELECT * FROM test_tab ORDER BY value; -- Test truncating a partition CREATE TABLE test_tab_part (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; -INSERT OVERWRITE TABLE test_tab_part PARTITION (part = '1') SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE test_tab_part PARTITION (part = '1') SELECT * FROM src tablesample (10 rows); DESC FORMATTED test_tab_part PARTITION (part = '1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q index a7aab357eaf2f..7a59efc4d1b01 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q @@ -2,9 +2,9 @@ CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; -INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 5; +INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (5 rows); -INSERT INTO TABLE test_tab SELECT * FROM src LIMIT 5; +INSERT INTO TABLE test_tab SELECT * FROM src tablesample (5 rows); -- The value should be 2 indicating the table has 2 files SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM test_tab; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q index c0e81e9ac051f..975c0f1ae8426 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q @@ -1,5 +1,5 @@ create table src_truncate (key string, value string); -load data local inpath '../data/files/kv1.txt' into table src_truncate;; +load data local inpath '../../data/files/kv1.txt' into table src_truncate;; create table srcpart_truncate (key string, value string) partitioned by (ds string, hr string); alter table srcpart_truncate add partition (ds='2008-04-08', hr='11'); @@ -7,10 +7,10 @@ alter table srcpart_truncate add partition (ds='2008-04-08', hr='12'); alter table srcpart_truncate add partition (ds='2008-04-09', hr='11'); alter table srcpart_truncate add partition (ds='2008-04-09', hr='12'); -load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='11'); -load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='12'); -load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='11'); -load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='12'); +load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='11'); +load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='12'); +load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='11'); +load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='12'); set hive.fetch.task.convertion=more; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q index a1607320c7a2d..4d1d978f829ee 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT IF(false, 1, cast(2 as smallint)) + 3 FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q index 63dd66ebedd0c..4c4a828fe0ee2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; -- casting from null should yield null select diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q index 0d36bc44fe08b..b18c01425c8c6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; -- Check for int, bigint automatic type widening conversions in UDFs, UNIONS EXPLAIN SELECT COALESCE(0, 9223372036854775807) FROM src LIMIT 1; SELECT COALESCE(0, 9223372036854775807) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q index 45aaa022dc8a4..04bea32101bdf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q @@ -1,6 +1,9 @@ DESCRIBE FUNCTION collect_set; DESCRIBE FUNCTION EXTENDED collect_set; +DESCRIBE FUNCTION collect_list; +DESCRIBE FUNCTION EXTENDED collect_list; + set hive.map.aggr = false; set hive.groupby.skewindata = false; @@ -8,6 +11,10 @@ SELECT key, collect_set(value) FROM src GROUP BY key ORDER BY key limit 20; +SELECT key, collect_list(value) +FROM src +GROUP BY key ORDER by key limit 20; + set hive.map.aggr = true; set hive.groupby.skewindata = false; @@ -15,6 +22,10 @@ SELECT key, collect_set(value) FROM src GROUP BY key ORDER BY key limit 20; +SELECT key, collect_list(value) +FROM src +GROUP BY key ORDER BY key limit 20; + set hive.map.aggr = false; set hive.groupby.skewindata = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q index dda7aaa5f2041..f065385688a1d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q @@ -1,5 +1,5 @@ CREATE TABLE kafka (contents STRING); -LOAD DATA LOCAL INPATH '../data/files/text-en.txt' INTO TABLE kafka; +LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka; set mapred.reduce.tasks=1; set hive.exec.reducers.max=1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q index 6cc9ce2630dd1..a2edec4d64e4c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q @@ -2,7 +2,7 @@ DROP TABLE covar_tab; CREATE TABLE covar_tab (a INT, b INT, c INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE +LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE INTO TABLE covar_tab; DESCRIBE FUNCTION corr; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q index 0f5d5f35bf02a..a9937bae3c21c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q @@ -2,7 +2,7 @@ DROP TABLE covar_tab; CREATE TABLE covar_tab (a INT, b INT, c INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE +LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE INTO TABLE covar_tab; DESCRIBE FUNCTION covar_pop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q index 72b9c4bd40049..2b50d8f238140 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q @@ -2,7 +2,7 @@ DROP TABLE covar_tab; CREATE TABLE covar_tab (a INT, b INT, c INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE +LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE INTO TABLE covar_tab; DESCRIBE FUNCTION covar_samp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q index 31ffd29a88acd..6a2fde52e42f6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q @@ -1,5 +1,5 @@ CREATE TABLE kafka (contents STRING); -LOAD DATA LOCAL INPATH '../data/files/text-en.txt' INTO TABLE kafka; +LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka; set mapred.reduce.tasks=1; set hive.exec.reducers.max=1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q new file mode 100644 index 0000000000000..8ebf01dcecb8c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q @@ -0,0 +1 @@ +select percentile(cast(key as bigint), 0.3) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q index 66c408d71bc1a..5b8ad7a08f5d3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q @@ -1,10 +1,10 @@ -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) CREATE TABLE bucket (key double, value string) CLUSTERED BY (key) SORTED BY (key DESC) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket; create table t1 (result double); create table t2 (result double); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q index 07bfb6e1fb2ab..1efa2951efd27 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q @@ -2,10 +2,10 @@ -- 0.23 changed input order of data in reducer task, which affects result of percentile_approx CREATE TABLE bucket (key double, value string) CLUSTERED BY (key) SORTED BY (key DESC) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket; create table t1 (result double); create table t2 (result double); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q new file mode 100644 index 0000000000000..0d86a42128d25 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q @@ -0,0 +1,6 @@ +-- HIVE-5279 +-- GenericUDAFSumList has Converter which does not have default constructor +-- After +create temporary function sum_list as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSumList'; + +select sum_list(array(key, key)) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q index 113af61062b0d..41bdec08278a8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q @@ -1,14 +1,16 @@ -explain -select E() FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select E() FROM src tablesample (1 rows); -select E() FROM src LIMIT 1; +select E() FROM src tablesample (1 rows); DESCRIBE FUNCTION E; DESCRIBE FUNCTION EXTENDED E; explain -select E() FROM src LIMIT 1; +select E() FROM src tablesample (1 rows); -select E() FROM src LIMIT 1; +select E() FROM src tablesample (1 rows); DESCRIBE FUNCTION E; DESCRIBE FUNCTION EXTENDED E; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q index 1fde7df5d251e..945483ecbfea8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q @@ -1,14 +1,16 @@ -explain -select PI() FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select PI() FROM src tablesample (1 rows); -select PI() FROM src LIMIT 1; +select PI() FROM src tablesample (1 rows); DESCRIBE FUNCTION PI; DESCRIBE FUNCTION EXTENDED PI; explain -select PI() FROM src LIMIT 1; +select PI() FROM src tablesample (1 rows); -select PI() FROM src LIMIT 1; +select PI() FROM src tablesample (1 rows); DESCRIBE FUNCTION PI; DESCRIBE FUNCTION EXTENDED PI; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q index f4f227d0dc168..0c06a5b6cd7f9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION abs; DESCRIBE FUNCTION EXTENDED abs; @@ -7,7 +9,7 @@ EXPLAIN SELECT abs(123), abs(-9223372036854775807), abs(9223372036854775807) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT abs(0), @@ -15,16 +17,16 @@ SELECT abs(123), abs(-9223372036854775807), abs(9223372036854775807) -FROM src LIMIT 1; +FROM src tablesample (1 rows); EXPLAIN SELECT abs(0.0), abs(-3.14159265), abs(3.14159265) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT abs(0.0), abs(-3.14159265), abs(3.14159265) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q index 625a2aa5c6aa6..f9adc16931dd6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q @@ -1,14 +1,16 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION acos; DESCRIBE FUNCTION EXTENDED acos; SELECT acos(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT acos(0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT acos(-0.5), asin(0.66) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT acos(2) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q index fca8fe8d1c00c..5a6a1830b1a51 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION array; DESCRIBE FUNCTION EXTENDED array; EXPLAIN SELECT array(), array()[1], array(1, 2, 3), array(1, 2, 3)[2], array(1,"a", 2, 3), array(1,"a", 2, 3)[2], -array(array(1), array(2), array(3), array(4))[1][0] FROM src LIMIT 1; +array(array(1), array(2), array(3), array(4))[1][0] FROM src tablesample (1 rows); SELECT array(), array()[1], array(1, 2, 3), array(1, 2, 3)[2], array(1,"a", 2, 3), array(1,"a", 2, 3)[2], -array(array(1), array(2), array(3), array(4))[1][0] FROM src LIMIT 1; +array(array(1), array(2), array(3), array(4))[1][0] FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q index 937bb0be03bd3..d2dad644065a2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q @@ -1,9 +1,11 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION array_contains; DESCRIBE FUNCTION EXTENDED array_contains; -- evalutes function for array of primitives -SELECT array_contains(array(1, 2, 3), 1) FROM src LIMIT 1; +SELECT array_contains(array(1, 2, 3), 1) FROM src tablesample (1 rows); -- evaluates function for nested arrays SELECT array_contains(array(array(1,2), array(2,3), array(3,4)), array(1,2)) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q index 53b389fd38fd8..3d885a2563520 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION ascii; DESCRIBE FUNCTION EXTENDED ascii; @@ -5,10 +7,10 @@ EXPLAIN SELECT ascii('Facebook'), ascii(''), ascii('!') -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT ascii('Facebook'), ascii(''), ascii('!') -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q index f95a5f57df8f8..73b77d10f0b69 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q @@ -1,14 +1,16 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION asin; DESCRIBE FUNCTION EXTENDED asin; SELECT asin(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT asin(0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT asin(-0.5), asin(0.66) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT asin(2) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q index d4ef03deb81b0..090438cb0f0cb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q @@ -1,16 +1,18 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION atan; DESCRIBE FUNCTION EXTENDED atan; SELECT atan(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT atan(1), atan(6), atan(-1.0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); DESCRIBE FUNCTION atan; DESCRIBE FUNCTION EXTENDED atan; SELECT atan(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT atan(1), atan(6), atan(-1.0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q index eb3ccea82e631..b22ee9c3cecf1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + describe function between; describe function extended between; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q index 2b9ad62a39dbe..c5a7ac1a60bcd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION bin; DESCRIBE FUNCTION EXTENDED bin; @@ -5,7 +7,7 @@ SELECT bin(1), bin(0), bin(99992421) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Negative numbers should be treated as two's complement (64 bit). -SELECT bin(-5) FROM src LIMIT 1; +SELECT bin(-5) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q index 7ea50dac31d07..ed7711cd6d5de 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q @@ -1,11 +1,13 @@ -select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src limit 1; -select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src limit 1; +set hive.fetch.task.conversion=more; + +select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows); +select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src tablesample (1 rows); drop table bitmap_test; create table bitmap_test (a array, b array); insert overwrite table bitmap_test -select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src limit 10; +select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src tablesample (10 rows); select ewah_bitmap_and(a,b) from bitmap_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q index 88e961683e289..142b248cdd250 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q @@ -1,3 +1,5 @@ -select ewah_bitmap_empty(array(13,2,4,8589934592,0,0)) from src limit 1; +set hive.fetch.task.conversion=more; -select ewah_bitmap_empty(array(13,2,4,8589934592,4096,0)) from src limit 1; +select ewah_bitmap_empty(array(13,2,4,8589934592,0,0)) from src tablesample (1 rows); + +select ewah_bitmap_empty(array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q index 0b71e681a53d5..00785b73faa05 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q @@ -1,11 +1,13 @@ -select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src limit 1; -select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src limit 1; +set hive.fetch.task.conversion=more; + +select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows); +select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src tablesample (1 rows); drop table bitmap_test; create table bitmap_test (a array, b array); insert overwrite table bitmap_test -select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src limit 10; +select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src tablesample (10 rows); select ewah_bitmap_or(a,b) from bitmap_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q index 4f71e70e1f3bf..43573bfb12fd8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION case; DESCRIBE FUNCTION EXTENDED case; @@ -27,7 +29,7 @@ SELECT CASE 1 WHEN 22 THEN 23 WHEN 21 THEN 24 END -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT CASE 1 WHEN 1 THEN 2 @@ -54,10 +56,27 @@ SELECT CASE 1 WHEN 22 THEN 23 WHEN 21 THEN 24 END -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- verify that short-circuiting is working correctly for CASE -- we should never get to the ELSE branch, which would raise an exception SELECT CASE 1 WHEN 1 THEN 'yo' ELSE reflect('java.lang.String', 'bogus', 1) END -FROM src LIMIT 1; +FROM src tablesample (1 rows); + +-- Allow compatible types in when/return type +SELECT CASE 1 + WHEN 1 THEN 123.0BD + ELSE 0.0BD + END, + CASE 1 + WHEN 1.0 THEN 123 + WHEN 2 THEN 1.0 + ELSE 222.02BD + END, + CASE 'abc' + WHEN cast('abc' as varchar(3)) THEN 'abcd' + WHEN 'efg' THEN cast('efgh' as varchar(10)) + ELSE cast('ijkl' as char(4)) + END +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q index 736bb053cddc7..2aa76f1f1d82e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT CASE src_thrift.lint[0] WHEN 0 THEN src_thrift.lint[0] + 1 @@ -14,7 +16,7 @@ SELECT CASE src_thrift.lint[0] WHEN '0' THEN src_thrift.lstring ELSE NULL END)[0] -FROM src_thrift LIMIT 3; +FROM src_thrift tablesample (3 rows); SELECT CASE src_thrift.lint[0] WHEN 0 THEN src_thrift.lint[0] + 1 @@ -31,4 +33,4 @@ SELECT CASE src_thrift.lint[0] WHEN '0' THEN src_thrift.lstring ELSE NULL END)[0] -FROM src_thrift LIMIT 3; +FROM src_thrift tablesample (3 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q index 48ca29cbc3ba6..d3c417babd466 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION coalesce; DESCRIBE FUNCTION EXTENDED coalesce; @@ -20,7 +22,7 @@ SELECT COALESCE(1), COALESCE(NULL, 2.0, 3.0), COALESCE(2.0, NULL, 3.0), COALESCE(IF(TRUE, NULL, 0), NULL) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT COALESCE(1), COALESCE(1, 2), @@ -40,7 +42,7 @@ SELECT COALESCE(1), COALESCE(NULL, 2.0, 3.0), COALESCE(2.0, NULL, 3.0), COALESCE(IF(TRUE, NULL, 0), NULL) -FROM src LIMIT 1; +FROM src tablesample (1 rows); EXPLAIN SELECT COALESCE(src_thrift.lint[1], 999), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q index 6c12f81304e46..c7983b8eb2050 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN CREATE TEMPORARY FUNCTION test_udf_get_java_string AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaString'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q index f642f6a2d00df..e35a1cfa170ca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION concat; DESCRIBE FUNCTION EXTENDED concat; @@ -12,4 +14,10 @@ SELECT concat(1, 2), concat(1), concat('1234', 'abc', 'extra argument') -FROM src LIMIT 1; +FROM src tablesample (1 rows); + +-- binary/mixed +SELECT + concat(cast('ab' as binary), cast('cd' as binary)), + concat('ab', cast('cd' as binary)) +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q index 6a0ce20dc37b7..538dfae06f20d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION concat_ws; DESCRIBE FUNCTION EXTENDED concat_ws; @@ -24,7 +26,7 @@ SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'), concat_ws('_', array('www', 'face'), array('book', 'com', '1234')), concat_ws('**', 'www', array('face'), array('book', 'com', '1234')), concat_ws('[]', array('www'), 'face', array('book', 'com', '1234')), - concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1; + concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows); SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'), concat_ws('-', 'www', array('face', 'book', 'com'), '1234'), @@ -32,7 +34,7 @@ SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'), concat_ws('_', array('www', 'face'), array('book', 'com', '1234')), concat_ws('**', 'www', array('face'), array('book', 'com', '1234')), concat_ws('[]', array('www'), 'face', array('book', 'com', '1234')), - concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1; + concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows); SELECT concat_ws(NULL, array('www', 'face', 'book', 'com'), '1234'), concat_ws(NULL, 'www', array('face', 'book', 'com'), '1234'), @@ -40,4 +42,4 @@ SELECT concat_ws(NULL, array('www', 'face', 'book', 'com'), '1234'), concat_ws(NULL, array('www', 'face'), array('book', 'com', '1234')), concat_ws(NULL, 'www', array('face'), array('book', 'com', '1234')), concat_ws(NULL, array('www'), 'face', array('book', 'com', '1234')), - concat_ws(NULL, array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1; + concat_ws(NULL, array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q index 212bcfb57938f..c6d6cf8600c8c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION conv; DESCRIBE FUNCTION EXTENDED conv; @@ -9,7 +11,7 @@ SELECT conv('22', 10, 10), conv('110011', 2, 16), conv('facebook', 36, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Test negative numbers. If to_base is positive, the number should be handled -- as a two's complement (64-bit) @@ -18,7 +20,7 @@ SELECT conv('1011', 2, -16), conv('-1', 10, 16), conv('-15', 10, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Test overflow. If a number is two large, the result should be -1 (if signed) -- or MAX_LONG (if unsigned) @@ -27,7 +29,7 @@ SELECT conv('9223372036854775807', 36, -16), conv('-9223372036854775807', 36, 16), conv('-9223372036854775807', 36, -16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Test with invalid input. If one of the bases is invalid, the result should -- be NULL. If there is an invalid digit in the number, the longest valid @@ -37,7 +39,7 @@ SELECT conv('131', 1, 5), conv('515', 5, 100), conv('10', -2, 2) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Perform the same tests with number arguments. @@ -45,31 +47,31 @@ SELECT conv(4521, 10, 36), conv(22, 10, 10), conv(110011, 2, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT conv(-641, 10, -10), conv(1011, 2, -16), conv(-1, 10, 16), conv(-15, 10, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT conv(9223372036854775807, 36, 16), conv(9223372036854775807, 36, -16), conv(-9223372036854775807, 36, 16), conv(-9223372036854775807, 36, -16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT conv(123455, 3, 10), conv(131, 1, 5), conv(515, 5, 100), conv('10', -2, 2) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Make sure that state is properly reset. SELECT conv(key, 10, 16), conv(key, 16, 10) -FROM src LIMIT 3; +FROM src tablesample (3 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q index 7887c4c2603f1..11ef8d7d87b28 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION cos; DESCRIBE FUNCTION EXTENDED cos; SELECT cos(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT cos(0.98), cos(1.57), cos(-0.5) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q new file mode 100644 index 0000000000000..4ada035d87726 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q @@ -0,0 +1,26 @@ +DESCRIBE FUNCTION current_database; + +explain +select current_database(); +select current_database(); + +create database xxx; +use xxx; + +explain +select current_database(); +select current_database(); + +set hive.fetch.task.conversion=more; + +use default; + +explain +select current_database(); +select current_database(); + +use xxx; + +explain +select current_database(); +select current_database(); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q index 014ca1c6e6afb..d5360fe3b22f7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q @@ -1,14 +1,16 @@ -explain -select degrees(PI()) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select degrees(PI()) FROM src tablesample (1 rows); -select degrees(PI()) FROM src LIMIT 1; +select degrees(PI()) FROM src tablesample (1 rows); DESCRIBE FUNCTION degrees; DESCRIBE FUNCTION EXTENDED degrees; explain -select degrees(PI()) FROM src LIMIT 1; +select degrees(PI()) FROM src tablesample (1 rows); -select degrees(PI()) FROM src LIMIT 1; +select degrees(PI()) FROM src tablesample (1 rows); DESCRIBE FUNCTION degrees; DESCRIBE FUNCTION EXTENDED degrees; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q index 4229e625a042a..b0d2844d42346 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION div; DESCRIBE FUNCTION EXTENDED div; -SELECT 3 DIV 2 FROM SRC LIMIT 1; +SELECT 3 DIV 2 FROM SRC tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q index dc4b2e7884d68..d36ba99118f51 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION /; DESCRIBE FUNCTION EXTENDED /; -SELECT 3 / 2 FROM SRC LIMIT 1; +SELECT 3 / 2 FROM SRC tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q index c32340ac89feb..fae764965a9a6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION elt; DESCRIBE FUNCTION EXTENDED elt; @@ -13,7 +15,7 @@ SELECT elt(2, 'abc', 'defg'), elt(null, 'abc', 'defg'), elt(0, 'abc', 'defg'), elt(3, 'abc', 'defg') -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT elt(2, 'abc', 'defg'), elt(3, 'aa', 'bb', 'cc', 'dd', 'ee', 'ff', 'gg'), @@ -26,4 +28,4 @@ SELECT elt(2, 'abc', 'defg'), elt(null, 'abc', 'defg'), elt(0, 'abc', 'defg'), elt(3, 'abc', 'defg') -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q index f5e9a7df4be00..ea9b18bf1fad2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q @@ -1,12 +1,14 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION =; DESCRIBE FUNCTION EXTENDED =; DESCRIBE FUNCTION ==; DESCRIBE FUNCTION EXTENDED ==; -SELECT true=false, false=true, false=false, true=true, NULL=NULL, true=NULL, NULL=true, false=NULL, NULL=false FROM src LIMIT 1; +SELECT true=false, false=true, false=false, true=true, NULL=NULL, true=NULL, NULL=true, false=NULL, NULL=false FROM src tablesample (1 rows); DESCRIBE FUNCTION <=>; DESCRIBE FUNCTION EXTENDED <=>; -SELECT true<=>false, false<=>true, false<=>false, true<=>true, NULL<=>NULL, true<=>NULL, NULL<=>true, false<=>NULL, NULL<=>false FROM src LIMIT 1; +SELECT true<=>false, false<=>true, false<=>false, true<=>true, NULL<=>NULL, true<=>NULL, NULL<=>true, false<=>NULL, NULL<=>false FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q index 19af288ff8404..ae651644a778a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q @@ -1,22 +1,24 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION explode; DESCRIBE FUNCTION EXTENDED explode; -EXPLAIN EXTENDED SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3; -EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3) a GROUP BY a.myCol; +EXPLAIN EXTENDED SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows); +EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows)) a GROUP BY a.myCol; -SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3; -SELECT explode(array(1,2,3)) AS (myCol) FROM src LIMIT 3; -SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3) a GROUP BY a.myCol; +SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows); +SELECT explode(array(1,2,3)) AS (myCol) FROM src tablesample (1 rows); +SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows)) a GROUP BY a.myCol; -EXPLAIN EXTENDED SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3; -EXPLAIN EXTENDED SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3) a GROUP BY a.key, a.val; +EXPLAIN EXTENDED SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows); +EXPLAIN EXTENDED SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows)) a GROUP BY a.key, a.val; -SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3; -SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3) a GROUP BY a.key, a.val; +SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows); +SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows)) a GROUP BY a.key, a.val; drop table lazy_array_map; create table lazy_array_map (map_col map, array_col array); -INSERT OVERWRITE TABLE lazy_array_map select map(1,'one',2,'two',3,'three'), array('100','200','300') FROM src LIMIT 1; +INSERT OVERWRITE TABLE lazy_array_map select map(1,'one',2,'two',3,'three'), array('100','200','300') FROM src tablesample (1 rows); SELECT array_col, myCol from lazy_array_map lateral view explode(array_col) X AS myCol; SELECT map_col, myKey, myValue from lazy_array_map lateral view explode(map_col) X AS myKey, myValue; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q index e995f5cf3c084..be92c024d3ed2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION field; DESCRIBE FUNCTION EXTENDED field; @@ -5,7 +7,7 @@ SELECT field("x", "a", "b", "c", "d"), field(NULL, "a", "b", "c", "d"), field(0, 1, 2, 3, 4) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT field("a", "a", "b", "c", "d"), @@ -13,7 +15,7 @@ SELECT field("c", "a", "b", "c", "d"), field("d", "a", "b", "c", "d"), field("d", "a", "b", NULL, "d") -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT field(1, 1, 2, 3, 4), @@ -21,11 +23,11 @@ SELECT field(3, 1, 2, 3, 4), field(4, 1, 2, 3, 4), field(4, 1, 2, NULL, 4) -FROM src LIMIT 1; +FROM src tablesample (1 rows); CREATE TABLE test_table(col1 STRING, col2 STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE test_table; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE test_table; select col1,col2, field("66",col1), @@ -42,7 +44,7 @@ from test_table where col1="86" or col1="66"; CREATE TABLE test_table1(col1 int, col2 string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE test_table1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE test_table1; select col1,col2, field(66,col1), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q index eac2c6e91c3d1..72c65b419808e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION find_in_set; DESCRIBE FUNCTION EXTENDED find_in_set; @@ -6,18 +8,18 @@ FROM src1 SELECT find_in_set(src1.key,concat(src1.key,',',src1.value)); FROM src1 SELECT find_in_set(src1.key,concat(src1.key,',',src1.value)); -SELECT find_in_set('ab','ab,abc,abcde') FROM src1 LIMIT 1; -SELECT find_in_set('ab','abc,ab,bbb') FROM src1 LIMIT 1; -SELECT find_in_set('ab','def,abc,ab') FROM src1 LIMIT 1; -SELECT find_in_set('ab','abc,abd,abf') FROM src1 LIMIT 1; -SELECT find_in_set(null,'a,b,c') FROM src1 LIMIT 1; -SELECT find_in_set('a',null) FROM src1 LIMIT 1; -SELECT find_in_set('', '') FROM src1 LIMIT 1; -SELECT find_in_set('',',') FROM src1 LIMIT 1; -SELECT find_in_set('','a,,b') FROM src1 LIMIT 1; -SELECT find_in_set('','a,b,') FROM src1 LIMIT 1; -SELECT find_in_set(',','a,b,d,') FROM src1 LIMIT 1; -SELECT find_in_set('a','') FROM src1 LIMIT 1; -SELECT find_in_set('a,','a,b,c,d') FROM src1 LIMIT 1; +SELECT find_in_set('ab','ab,abc,abcde') FROM src1 tablesample (1 rows); +SELECT find_in_set('ab','abc,ab,bbb') FROM src1 tablesample (1 rows); +SELECT find_in_set('ab','def,abc,ab') FROM src1 tablesample (1 rows); +SELECT find_in_set('ab','abc,abd,abf') FROM src1 tablesample (1 rows); +SELECT find_in_set(null,'a,b,c') FROM src1 tablesample (1 rows); +SELECT find_in_set('a',null) FROM src1 tablesample (1 rows); +SELECT find_in_set('', '') FROM src1 tablesample (1 rows); +SELECT find_in_set('',',') FROM src1 tablesample (1 rows); +SELECT find_in_set('','a,,b') FROM src1 tablesample (1 rows); +SELECT find_in_set('','a,b,') FROM src1 tablesample (1 rows); +SELECT find_in_set(',','a,b,d,') FROM src1 tablesample (1 rows); +SELECT find_in_set('a','') FROM src1 tablesample (1 rows); +SELECT find_in_set('a,','a,b,c,d') FROM src1 tablesample (1 rows); SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,2,956')=0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q index e2084cddf0ec7..2504bd0b68306 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + use default; -- Test format_number() UDF @@ -7,26 +9,26 @@ DESCRIBE FUNCTION EXTENDED format_number; EXPLAIN SELECT format_number(12332.123456, 4), format_number(12332.1,4), - format_number(12332.2,0) FROM src limit 1; + format_number(12332.2,0) FROM src tablesample (1 rows); SELECT format_number(12332.123456, 4), format_number(12332.1,4), format_number(12332.2,0) -FROM src limit 1; +FROM src tablesample (1 rows); -- positive numbers SELECT format_number(0.123456789, 12), format_number(12345678.123456789, 5), format_number(1234567.123456789, 7), format_number(123456.123456789, 0) -FROM src limit 1; +FROM src tablesample (1 rows); -- negative numbers SELECT format_number(-123456.123456789, 0), format_number(-1234567.123456789, 2), format_number(-0.123456789, 15), format_number(-12345.123456789, 4) -FROM src limit 1; +FROM src tablesample (1 rows); -- zeros SELECT format_number(0.0, 4), @@ -34,7 +36,7 @@ SELECT format_number(0.0, 4), format_number(000.0000, 1), format_number(00000.0000, 1), format_number(-00.0, 4) -FROM src limit 1; +FROM src tablesample (1 rows); -- integers SELECT format_number(0, 0), @@ -42,7 +44,7 @@ SELECT format_number(0, 0), format_number(12, 2), format_number(123, 5), format_number(1234, 7) -FROM src limit 1; +FROM src tablesample (1 rows); -- long and double boundary -- 9223372036854775807 is LONG_MAX @@ -54,4 +56,4 @@ SELECT format_number(-9223372036854775807, 10), format_number(9223372036854775807, 20), format_number(4.9E-324, 324), format_number(1.7976931348623157E308, 308) -FROM src limit 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q index 464f2df3dcd7a..05f7f5a9811bc 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION get_json_object; DESCRIBE FUNCTION EXTENDED get_json_object; @@ -5,6 +7,8 @@ CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; FROM src INSERT OVERWRITE TABLE dest1 SELECT ' abc ' WHERE src.key = 86; +set hive.fetch.task.conversion=more; + EXPLAIN SELECT get_json_object(src_json.json, '$.owner') FROM src_json; @@ -33,8 +37,8 @@ SELECT get_json_object(src_json.json, '$.fb:testid') FROM src_json; CREATE TABLE dest2(c1 STRING) STORED AS RCFILE; -INSERT OVERWRITE TABLE dest2 SELECT '{"a":"b\nc"}' FROM src LIMIT 1; +INSERT OVERWRITE TABLE dest2 SELECT '{"a":"b\nc"}' FROM src tablesample (1 rows); SELECT * FROM dest2; -SELECT get_json_object(c1, '$.a') FROM dest2; \ No newline at end of file +SELECT get_json_object(c1, '$.a') FROM dest2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q index aea110a9431cf..230bd244ffbc7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION >; DESCRIBE FUNCTION EXTENDED >; -SELECT true>false, false>true, false>false, true>true FROM src LIMIT 1; \ No newline at end of file +SELECT true>false, false>true, false>false, true>true FROM src tablesample (1 rows); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q index 8de165b9ea9ed..025eed7dd5582 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION >=; DESCRIBE FUNCTION EXTENDED >=; -SELECT true>=false, false>=true, false>=false, true>=true FROM src LIMIT 1; \ No newline at end of file +SELECT true>=false, false>=true, false>=false, true>=true FROM src tablesample (1 rows); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q index faf372218a107..5814a1779bd16 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION hash; DESCRIBE FUNCTION EXTENDED hash; @@ -7,11 +9,11 @@ SELECT hash(CAST(1 AS TINYINT)), hash(CAST(2 AS SMALLINT)), hash(CAST(1.25 AS FLOAT)), hash(CAST(16.0 AS DOUBLE)), hash('400'), hash('abc'), hash(TRUE), hash(FALSE), hash(1, 2, 3) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT hash(CAST(1 AS TINYINT)), hash(CAST(2 AS SMALLINT)), hash(3), hash(CAST('123456789012' AS BIGINT)), hash(CAST(1.25 AS FLOAT)), hash(CAST(16.0 AS DOUBLE)), hash('400'), hash('abc'), hash(TRUE), hash(FALSE), hash(1, 2, 3) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q index 37e035ad42b00..0e5457965a71e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION hex; DESCRIBE FUNCTION EXTENDED hex; @@ -7,14 +9,14 @@ SELECT hex('Facebook'), hex('\0'), hex('qwertyuiopasdfghjkl') -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- If the argument is a number, hex should convert it to hexadecimal. SELECT hex(1), hex(0), hex(4207849477) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Negative numbers should be treated as two's complement (64 bit). -SELECT hex(-5) FROM src LIMIT 1; +SELECT hex(-5) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q index c5c366daa8c83..b9811e6c6f652 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION hour; DESCRIBE FUNCTION EXTENDED hour; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q index 4f7c8b4a36bad..d9285ff7ce19d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION if; DESCRIBE FUNCTION EXTENDED if; @@ -8,7 +10,7 @@ SELECT IF(TRUE, 1, 2) AS COL1, IF(2=2, 1, NULL) AS COL4, IF(2=2, NULL, 1) AS COL5, IF(IF(TRUE, NULL, FALSE), 1, 2) AS COL6 -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT IF(TRUE, 1, 2) AS COL1, @@ -17,7 +19,7 @@ SELECT IF(TRUE, 1, 2) AS COL1, IF(2=2, 1, NULL) AS COL4, IF(2=2, NULL, 1) AS COL5, IF(IF(TRUE, NULL, FALSE), 1, 2) AS COL6 -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Type conversions EXPLAIN @@ -25,10 +27,10 @@ SELECT IF(TRUE, CAST(128 AS SMALLINT), CAST(1 AS TINYINT)) AS COL1, IF(FALSE, 1, 1.1) AS COL2, IF(FALSE, 1, 'ABC') AS COL3, IF(FALSE, 'ABC', 12.3) AS COL4 -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT IF(TRUE, CAST(128 AS SMALLINT), CAST(1 AS TINYINT)) AS COL1, IF(FALSE, 1, 1.1) AS COL2, IF(FALSE, 1, 'ABC') AS COL3, IF(FALSE, 'ABC', 12.3) AS COL4 -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q index 75778138742d5..a7ce3c6f0bdbb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + SELECT 1 IN (1, 2, 3), 4 IN (1, 2, 3), array(1,2,3) IN (array(1,2,3)), @@ -8,6 +10,6 @@ SELECT 1 IN (1, 2, 3), 1 IN (1, 2, 3) OR false IN(false), NULL IN (1, 2, 3), 4 IN (1, 2, 3, NULL), - (1+3) IN (5, 6, (1+2) + 1) FROM src LIMIT 1; + (1+3) IN (5, 6, (1+2) + 1) FROM src tablesample (1 rows); SELECT key FROM src WHERE key IN ("238", 86); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q index 4da478908ddbd..9d9efe8e23d6e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q @@ -1,12 +1,12 @@ DESCRIBE FUNCTION in_file; EXPLAIN -SELECT in_file("303", "../data/files/test2.dat"), - in_file("304", "../data/files/test2.dat"), - in_file(CAST(NULL AS STRING), "../data/files/test2.dat") +SELECT in_file("303", "../../data/files/test2.dat"), + in_file("304", "../../data/files/test2.dat"), + in_file(CAST(NULL AS STRING), "../../data/files/test2.dat") FROM src LIMIT 1; -SELECT in_file("303", "../data/files/test2.dat"), - in_file("304", "../data/files/test2.dat"), - in_file(CAST(NULL AS STRING), "../data/files/test2.dat") +SELECT in_file("303", "../../data/files/test2.dat"), + in_file("304", "../../data/files/test2.dat"), + in_file(CAST(NULL AS STRING), "../../data/files/test2.dat") FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q index 9079d0e5743f6..6844f9d0bfc6c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q @@ -1,2 +1,3 @@ +set hive.support.quoted.identifiers=none; DESCRIBE FUNCTION `index`; DESCRIBE FUNCTION EXTENDED `index`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q index 39006f4b9b117..95d55f71c11cd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + describe function inline; explain SELECT inline( diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q index 20ed8e4ea0176..790a1049d1777 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION instr; DESCRIBE FUNCTION EXTENDED instr; @@ -15,7 +17,7 @@ SELECT instr('abcd', 'abc'), instr(CAST(16.0 AS DOUBLE), '.0'), instr(null, 'abc'), instr('abcd', null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT instr('abcd', 'abc'), instr('abcabc', 'ccc'), @@ -30,4 +32,4 @@ SELECT instr('abcd', 'abc'), instr(CAST(16.0 AS DOUBLE), '.0'), instr(null, 'abc'), instr('abcd', null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q index d1569cc7f6dbb..efb834efdc64f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION isnull; DESCRIBE FUNCTION EXTENDED isnull; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q index 2f28be1e9d0e1..51280b2567cca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION java_method; DESCRIBE FUNCTION EXTENDED java_method; @@ -11,7 +13,7 @@ SELECT java_method("java.lang.String", "valueOf", 1), java_method("java.lang.Math", "round", 2.5), java_method("java.lang.Math", "exp", 1.0), java_method("java.lang.Math", "floor", 1.9) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT java_method("java.lang.String", "valueOf", 1), @@ -21,5 +23,5 @@ SELECT java_method("java.lang.String", "valueOf", 1), java_method("java.lang.Math", "round", 2.5), java_method("java.lang.Math", "exp", 1.0), java_method("java.lang.Math", "floor", 1.9) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q index b84307970d630..4413751ae647b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION length; DESCRIBE FUNCTION EXTENDED length; @@ -9,6 +11,6 @@ DROP TABLE dest1; -- Test with non-ascii characters. CREATE TABLE dest1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE dest1; +LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE dest1; EXPLAIN SELECT length(dest1.name) FROM dest1; SELECT length(dest1.name) FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q index a2577006a885d..03326777bfa9f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION <; DESCRIBE FUNCTION EXTENDED <; -SELECT true; DESCRIBE FUNCTION EXTENDED <>; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q index 88396545de933..dceab7edaa1fa 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + SELECT 1 NOT IN (1, 2, 3), 4 NOT IN (1, 2, 3), 1 = 2 NOT IN (true, false), @@ -7,4 +9,4 @@ SELECT 1 NOT IN (1, 2, 3), "abc" NOT RLIKE "^bc", "abc" NOT REGEXP "^ab", "abc" NOT REGEXP "^bc", - 1 IN (1, 2) AND "abc" NOT LIKE "bc%" FROM src LIMIT 1; + 1 IN (1, 2) AND "abc" NOT LIKE "bc%" FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q index 0133b4b90bcbc..97162576df167 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; DESCRIBE FUNCTION nvl; DESCRIBE FUNCTION EXTENDED nvl; @@ -5,9 +6,9 @@ DESCRIBE FUNCTION EXTENDED nvl; EXPLAIN SELECT NVL( 1 , 2 ) AS COL1, NVL( NULL, 5 ) AS COL2 -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT NVL( 1 , 2 ) AS COL1, NVL( NULL, 5 ) AS COL2 -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q index 9ff73d42b41e6..d42a2f337ebca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q @@ -1,20 +1,22 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION pmod; DESCRIBE FUNCTION EXTENDED pmod; SELECT pmod(null, null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT pmod(-100,9), pmod(-50,101), pmod(-1000,29) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT pmod(100,19), pmod(50,125), pmod(300,15) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -SELECT pmod(CAST(-100 AS TINYINT),CAST(9 AS TINYINT)), pmod(CAST(-50 AS TINYINT),CAST(101 AS TINYINT)), pmod(CAST(-100 AS TINYINT),CAST(29 AS TINYINT)) FROM src LIMIT 1; -SELECT pmod(CAST(-100 AS SMALLINT),CAST(9 AS SMALLINT)), pmod(CAST(-50 AS SMALLINT),CAST(101 AS SMALLINT)), pmod(CAST(-100 AS SMALLINT),CAST(29 AS SMALLINT)) FROM src LIMIT 1; -SELECT pmod(CAST(-100 AS BIGINT),CAST(9 AS BIGINT)), pmod(CAST(-50 AS BIGINT),CAST(101 AS BIGINT)), pmod(CAST(-100 AS BIGINT),CAST(29 AS BIGINT)) FROM src LIMIT 1; +SELECT pmod(CAST(-100 AS TINYINT),CAST(9 AS TINYINT)), pmod(CAST(-50 AS TINYINT),CAST(101 AS TINYINT)), pmod(CAST(-100 AS TINYINT),CAST(29 AS TINYINT)) FROM src tablesample (1 rows); +SELECT pmod(CAST(-100 AS SMALLINT),CAST(9 AS SMALLINT)), pmod(CAST(-50 AS SMALLINT),CAST(101 AS SMALLINT)), pmod(CAST(-100 AS SMALLINT),CAST(29 AS SMALLINT)) FROM src tablesample (1 rows); +SELECT pmod(CAST(-100 AS BIGINT),CAST(9 AS BIGINT)), pmod(CAST(-50 AS BIGINT),CAST(101 AS BIGINT)), pmod(CAST(-100 AS BIGINT),CAST(29 AS BIGINT)) FROM src tablesample (1 rows); -SELECT pmod(CAST(-100.91 AS FLOAT),CAST(9.8 AS FLOAT)), pmod(CAST(-50.1 AS FLOAT),CAST(101.8 AS FLOAT)), pmod(CAST(-100.91 AS FLOAT),CAST(29.75 AS FLOAT)) FROM src LIMIT 1; -SELECT pmod(CAST(-100.91 AS DOUBLE),CAST(9.8 AS DOUBLE)), pmod(CAST(-50.1 AS DOUBLE),CAST(101.8 AS DOUBLE)), pmod(CAST(-100.91 AS DOUBLE),CAST(29.75 AS DOUBLE)) FROM src LIMIT 1; -SELECT pmod(CAST(-100.91 AS DECIMAL),CAST(9.8 AS DECIMAL)), pmod(CAST(-50.1 AS DECIMAL),CAST(101.8 AS DECIMAL)), pmod(CAST(-100.91 AS DECIMAL),CAST(29.75 AS DECIMAL)) FROM src LIMIT 1; +SELECT pmod(CAST(-100.91 AS FLOAT),CAST(9.8 AS FLOAT)), pmod(CAST(-50.1 AS FLOAT),CAST(101.8 AS FLOAT)), pmod(CAST(-100.91 AS FLOAT),CAST(29.75 AS FLOAT)) FROM src tablesample (1 rows); +SELECT pmod(CAST(-100.91 AS DOUBLE),CAST(9.8 AS DOUBLE)), pmod(CAST(-50.1 AS DOUBLE),CAST(101.8 AS DOUBLE)), pmod(CAST(-100.91 AS DOUBLE),CAST(29.75 AS DOUBLE)) FROM src tablesample (1 rows); +SELECT pmod(CAST(-100.91 AS DECIMAL(5,2)),CAST(9.8 AS DECIMAL(2,1))), pmod(CAST(-50.1 AS DECIMAL(3,1)),CAST(101.8 AS DECIMAL(4,1))), pmod(CAST(-100.91 AS DECIMAL(5,2)),CAST(29.75 AS DECIMAL(4,2))) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q index 99e89ccbf773a..115e4e56f1806 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q @@ -4,21 +4,23 @@ use default; DESCRIBE FUNCTION printf; DESCRIBE FUNCTION EXTENDED printf; +set hive.fetch.task.conversion=more; + EXPLAIN -SELECT printf("Hello World %d %s", 100, "days") FROM src LIMIT 1; +SELECT printf("Hello World %d %s", 100, "days") FROM src tablesample (1 rows); -- Test Primitive Types -SELECT printf("Hello World %d %s", 100, "days") FROM src LIMIT 1; -SELECT printf("All Type Test: %b, %c, %d, %e, %+10.4f, %g, %h, %s, %a", false, 65, 15000, 12.3400, 27183.240051, 2300.41, 50, "corret", 256.125) FROM src LIMIT 1; +SELECT printf("Hello World %d %s", 100, "days") FROM src tablesample (1 rows); +SELECT printf("All Type Test: %b, %c, %d, %e, %+10.4f, %g, %h, %s, %a", false, 65, 15000, 12.3400, 27183.240051, 2300.41, 50, "corret", 256.125) FROM src tablesample (1 rows); -- Test NULL Values -SELECT printf("Color %s, String Null: %s, number1 %d, number2 %05d, Integer Null: %d, hex %#x, float %5.2f Double Null: %f\n", "red", NULL, 123456, 89, NULL, 255, 3.14159, NULL) FROM src LIMIT 1; +SELECT printf("Color %s, String Null: %s, number1 %d, number2 %05d, Integer Null: %d, hex %#x, float %5.2f Double Null: %f\n", "red", NULL, 123456, 89, NULL, 255, 3.14159, NULL) FROM src tablesample (1 rows); -- Test Timestamp create table timestamp_udf (t timestamp); -from src +from (select * from src tablesample (1 rows)) s insert overwrite table timestamp_udf - select '2011-05-06 07:08:09.1234567' limit 1; + select '2011-05-06 07:08:09.1234567'; select printf("timestamp: %s", t) from timestamp_udf; drop table timestamp_udf; @@ -27,7 +29,7 @@ CREATE TABLE binay_udf(key binary, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '9' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE binay_udf; +LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE binay_udf; create table dest1 (key binary, value int); insert overwrite table dest1 select transform(*) using 'cat' as key binary, value int from binay_udf; select value, printf("format key: %s", key) from dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q index 001d1cf8b6464..19242bd757b41 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q @@ -1,16 +1,18 @@ -explain -select radians(57.2958) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select radians(57.2958) FROM src tablesample (1 rows); -select radians(57.2958) FROM src LIMIT 1; -select radians(143.2394) FROM src LIMIT 1; +select radians(57.2958) FROM src tablesample (1 rows); +select radians(143.2394) FROM src tablesample (1 rows); DESCRIBE FUNCTION radians; DESCRIBE FUNCTION EXTENDED radians; explain -select radians(57.2958) FROM src LIMIT 1; +select radians(57.2958) FROM src tablesample (1 rows); -select radians(57.2958) FROM src LIMIT 1; -select radians(143.2394) FROM src LIMIT 1; +select radians(57.2958) FROM src tablesample (1 rows); +select radians(143.2394) FROM src tablesample (1 rows); DESCRIBE FUNCTION radians; DESCRIBE FUNCTION EXTENDED radians; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q index f357ff5049564..cef1e4a5d9195 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION reflect; DESCRIBE FUNCTION EXTENDED reflect; @@ -10,7 +12,7 @@ SELECT reflect("java.lang.String", "valueOf", 1), reflect("java.lang.Math", "exp", 1.0), reflect("java.lang.Math", "floor", 1.9), reflect("java.lang.Integer", "valueOf", key, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT reflect("java.lang.String", "valueOf", 1), @@ -21,4 +23,4 @@ SELECT reflect("java.lang.String", "valueOf", 1), reflect("java.lang.Math", "exp", 1.0), reflect("java.lang.Math", "floor", 1.9), reflect("java.lang.Integer", "valueOf", key, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q index 9ffd755292f66..a65294b335844 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION reflect2; DESCRIBE FUNCTION EXTENDED reflect2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q index 3aee10963d747..12b685b32c69c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q @@ -1,6 +1,8 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION regexp; DESCRIBE FUNCTION EXTENDED regexp; SELECT 'fofo' REGEXP '^fo', 'fo\no' REGEXP '^fo\no$', 'Bn' REGEXP '^Ba*n', 'afofo' REGEXP 'fo', 'afofo' REGEXP '^fo', 'Baan' REGEXP '^Ba?n', 'axe' REGEXP 'pi|apa', 'pip' REGEXP '^(pi)*$' -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q index 162085f4c71c8..91474bac2a16b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION repeat; DESCRIBE FUNCTION EXTENDED repeat; @@ -6,11 +8,11 @@ EXPLAIN SELECT repeat("", 4), repeat("asd", 0), repeat("asdf", -1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT repeat("Facebook", 3), repeat("", 4), repeat("asd", 0), repeat("asdf", -1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q index 81f765ec5937c..89aafe3443eeb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q @@ -11,5 +11,5 @@ DROP TABLE dest1; -- kv4.txt contains the text 0xE982B5E993AE, which should be reversed to -- 0xE993AEE982B5 CREATE TABLE dest1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE dest1; +LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE dest1; SELECT count(1) FROM dest1 WHERE reverse(dest1.name) = _UTF-8 0xE993AEE982B5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q index 18ebba8708493..88b22749a3095 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q @@ -1,15 +1,17 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION round; DESCRIBE FUNCTION EXTENDED round; SELECT round(null), round(null, 0), round(125, null), round(1.0/0.0, 0), round(power(-1.0,0.5), 0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT round(55555), round(55555, 0), round(55555, 1), round(55555, 2), round(55555, 3), round(55555, -1), round(55555, -2), round(55555, -3), round(55555, -4), round(55555, -5), round(55555, -6), round(55555, -7), round(55555, -8) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT round(125.315), round(125.315, 0), @@ -18,7 +20,7 @@ SELECT round(-125.315), round(-125.315, 0), round(-125.315, 1), round(-125.315, 2), round(-125.315, 3), round(-125.315, 4), round(-125.315, -1), round(-125.315, -2), round(-125.315, -3), round(-125.315, -4) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT round(3.141592653589793, -15), round(3.141592653589793, -16), @@ -38,7 +40,7 @@ SELECT round(3.141592653589793, 12), round(3.141592653589793, 13), round(3.141592653589793, 13), round(3.141592653589793, 14), round(3.141592653589793, 15), round(3.141592653589793, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -SELECT round(1809242.3151111344, 9), round(-1809242.3151111344, 9) -FROM src LIMIT 1; +SELECT round(1809242.3151111344, 9), round(-1809242.3151111344, 9), round(1809242.3151111344BD, 9), round(-1809242.3151111344BD, 9) +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q index 6be30855aea1b..43988c1225cd8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q @@ -1,8 +1,10 @@ --- test for NaN (not-a-number) +set hive.fetch.task.conversion=more; + +-- test for NaN (not-a-number) create table tstTbl1(n double); insert overwrite table tstTbl1 -select 'NaN' from src limit 1; +select 'NaN' from src tablesample (1 rows); select * from tstTbl1; @@ -10,4 +12,4 @@ select round(n, 1) from tstTbl1; select round(n) from tstTbl1; -- test for Infinity -select round(1/0), round(1/0, 2), round(1.0/0.0), round(1.0/0.0, 2) from src limit 1; +select round(1/0), round(1/0, 2), round(1.0/0.0), round(1.0/0.0, 2) from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q index 50a1f44419047..f042b6f3fa9c7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q @@ -1,14 +1,16 @@ +set hive.fetch.task.conversion=more; + -- test for TINYINT -select round(-128), round(127), round(0) from src limit 1; +select round(-128), round(127), round(0) from src tablesample (1 rows); -- test for SMALLINT -select round(-32768), round(32767), round(-129), round(128) from src limit 1; +select round(-32768), round(32767), round(-129), round(128) from src tablesample (1 rows); -- test for INT -select round(cast(negative(pow(2, 31)) as INT)), round(cast((pow(2, 31) - 1) as INT)), round(-32769), round(32768) from src limit 1; +select round(cast(negative(pow(2, 31)) as INT)), round(cast((pow(2, 31) - 1) as INT)), round(-32769), round(32768) from src tablesample (1 rows); -- test for BIGINT -select round(cast(negative(pow(2, 63)) as BIGINT)), round(cast((pow(2, 63) - 1) as BIGINT)), round(cast(negative(pow(2, 31) + 1) as BIGINT)), round(cast(pow(2, 31) as BIGINT)) from src limit 1; +select round(cast(negative(pow(2, 63)) as BIGINT)), round(cast((pow(2, 63) - 1) as BIGINT)), round(cast(negative(pow(2, 31) + 1) as BIGINT)), round(cast(pow(2, 31) as BIGINT)) from src tablesample (1 rows); -- test for DOUBLE -select round(126.1), round(126.7), round(32766.1), round(32766.7) from src limit 1; +select round(126.1), round(126.7), round(32766.1), round(32766.7) from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q index 01e5fbd429b17..4ee69e8985075 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION rpad; DESCRIBE FUNCTION EXTENDED rpad; @@ -5,10 +7,10 @@ EXPLAIN SELECT rpad('hi', 1, '?'), rpad('hi', 5, '.'), rpad('hi', 6, '123') -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT rpad('hi', 1, '?'), rpad('hi', 5, '.'), rpad('hi', 6, '123') -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q index 1943188086ea5..f63426d7e7257 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION second; DESCRIBE FUNCTION EXTENDED second; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q index abceb3439ec1e..b1602e8a257ef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q @@ -1,20 +1,22 @@ -explain -select sign(0) FROM src LIMIT 1; -select sign(0) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select sign(0) FROM src tablesample (1 rows); +select sign(0) FROM src tablesample (1 rows); -select sign(-45) FROM src LIMIT 1; +select sign(-45) FROM src tablesample (1 rows); -select sign(46) FROM src LIMIT 1; +select sign(46) FROM src tablesample (1 rows); DESCRIBE FUNCTION sign; DESCRIBE FUNCTION EXTENDED sign; explain -select sign(0) FROM src LIMIT 1; -select sign(0) FROM src LIMIT 1; +select sign(0) FROM src tablesample (1 rows); +select sign(0) FROM src tablesample (1 rows); -select sign(-45) FROM src LIMIT 1; +select sign(-45) FROM src tablesample (1 rows); -select sign(46) FROM src LIMIT 1; +select sign(46) FROM src tablesample (1 rows); DESCRIBE FUNCTION sign; DESCRIBE FUNCTION EXTENDED sign; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q index abb7cac8dac9d..79745be772f6f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION sin; DESCRIBE FUNCTION EXTENDED sin; SELECT sin(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT sin(0.98), sin(1.57), sin(-0.5) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q index 8aaa68a6e318a..f6f76a30e7258 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION size; DESCRIBE FUNCTION EXTENDED size; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q index ef0973212a27c..313bcf8a1ea52 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + use default; -- Test sort_array() UDF @@ -6,16 +8,16 @@ DESCRIBE FUNCTION EXTENDED sort_array; -- Evaluate function against STRING valued keys EXPLAIN -SELECT sort_array(array("b", "d", "c", "a")) FROM src LIMIT 1; +SELECT sort_array(array("b", "d", "c", "a")) FROM src tablesample (1 rows); -SELECT sort_array(array("f", "a", "g", "c", "b", "d", "e")) FROM src LIMIT 1; -SELECT sort_array(sort_array(array("hadoop distributed file system", "enterprise databases", "hadoop map-reduce"))) FROM src LIMIT 1; +SELECT sort_array(array("f", "a", "g", "c", "b", "d", "e")) FROM src tablesample (1 rows); +SELECT sort_array(sort_array(array("hadoop distributed file system", "enterprise databases", "hadoop map-reduce"))) FROM src tablesample (1 rows); -- Evaluate function against INT valued keys -SELECT sort_array(array(2, 9, 7, 3, 5, 4, 1, 6, 8)) FROM src LIMIT 1; +SELECT sort_array(array(2, 9, 7, 3, 5, 4, 1, 6, 8)) FROM src tablesample (1 rows); -- Evaluate function against FLOAT valued keys -SELECT sort_array(sort_array(array(2.333, 9, 1.325, 2.003, 0.777, -3.445, 1))) FROM src LIMIT 1; +SELECT sort_array(sort_array(array(2.333, 9, 1.325, 2.003, 0.777, -3.445, 1))) FROM src tablesample (1 rows); -- Test it against data in a table. CREATE TABLE dest1 ( @@ -30,7 +32,7 @@ CREATE TABLE dest1 ( timestamps ARRAY ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/primitive_type_arrays.txt' OVERWRITE INTO TABLE dest1; +LOAD DATA LOCAL INPATH '../../data/files/primitive_type_arrays.txt' OVERWRITE INTO TABLE dest1; SELECT sort_array(tinyints), sort_array(smallints), sort_array(ints), sort_array(bigints), sort_array(booleans), sort_array(floats), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q index cf6466fb63bad..cc616f784fcea 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION space; DESCRIBE FUNCTION EXTENDED space; @@ -7,7 +9,7 @@ EXPLAIN SELECT space(1), space(-1), space(-100) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT length(space(10)), @@ -15,7 +17,7 @@ SELECT length(space(1)), length(space(-1)), length(space(-100)) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT space(10), @@ -23,5 +25,5 @@ SELECT space(1), space(-1), space(-100) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q index f79901736cf7c..55919eac743b7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION split; DESCRIBE FUNCTION EXTENDED split; @@ -6,11 +8,11 @@ EXPLAIN SELECT split('oneAtwoBthreeC', '[ABC]'), split('', '.'), split(50401020, 0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT split('a b c', ' '), split('oneAtwoBthreeC', '[ABC]'), split('', '.'), split(50401020, 0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q index 3ee19c83699bb..ee2135b509ae2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q @@ -1,9 +1,11 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION struct; DESCRIBE FUNCTION EXTENDED struct; EXPLAIN SELECT struct(1), struct(1, "a"), struct(1, "b", 1.5).col1, struct(1, struct("a", 1.5)).col2.col1 -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT struct(1), struct(1, "a"), struct(1, "b", 1.5).col1, struct(1, struct("a", 1.5)).col2.col1 -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q index 32757bef51c1c..2d04f904bbc6f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION substr; DESCRIBE FUNCTION EXTENDED substr; @@ -5,7 +7,7 @@ SELECT substr(null, 1), substr(null, 1, 1), substr('ABC', null), substr('ABC', null, 1), substr('ABC', 1, null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT substr('ABC', 1, 0), substr('ABC', 1, -1), substr('ABC', 2, -100), @@ -14,7 +16,7 @@ SELECT substr('ABC', 100), substr('ABC', 100, 100), substr('ABC', -100), substr('ABC', -100, 100), substr('ABC', 2147483647), substr('ABC', 2147483647, 2147483647) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT substr('ABCDEFG', 3, 4), substr('ABCDEFG', -5, 4), @@ -22,7 +24,7 @@ SELECT substr('ABC', 0), substr('ABC', 1), substr('ABC', 2), substr('ABC', 3), substr('ABC', 1, 2147483647), substr('ABC', 2, 2147483647), substr('A', 0), substr('A', 1), substr('A', -1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT substr('ABC', 0, 1), substr('ABC', 0, 2), substr('ABC', 0, 3), substr('ABC', 0, 4), @@ -30,14 +32,14 @@ SELECT substr('ABC', 2, 1), substr('ABC', 2, 2), substr('ABC', 2, 3), substr('ABC', 2, 4), substr('ABC', 3, 1), substr('ABC', 3, 2), substr('ABC', 3, 3), substr('ABC', 3, 4), substr('ABC', 4, 1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT substr('ABC', -1, 1), substr('ABC', -1, 2), substr('ABC', -1, 3), substr('ABC', -1, 4), substr('ABC', -2, 1), substr('ABC', -2, 2), substr('ABC', -2, 3), substr('ABC', -2, 4), substr('ABC', -3, 1), substr('ABC', -3, 2), substr('ABC', -3, 3), substr('ABC', -3, 4), substr('ABC', -4, 1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- substring() is a synonim of substr(), so just perform some basic tests SELECT @@ -46,7 +48,7 @@ SELECT substring('ABC', 0), substring('ABC', 1), substring('ABC', 2), substring('ABC', 3), substring('ABC', 1, 2147483647), substring('ABC', 2, 2147483647), substring('A', 0), substring('A', 1), substring('A', -1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- test for binary substr SELECT @@ -63,7 +65,7 @@ SELECT substr(ABC, -3, 1), substr(ABC, -3, 2), substr(ABC, -3, 3), substr(ABC, -3, 4), substr(ABC, -4, 1) FROM ( - select CAST(concat(substr(value, 1, 0), 'ABC') as BINARY) as ABC from src LIMIT 1 + select CAST(concat(substr(value, 1, 0), 'ABC') as BINARY) as ABC from src tablesample (1 rows) ) X; -- test UTF-8 substr @@ -72,4 +74,4 @@ SELECT substr("abc 玩", 5), substr("abc 玩玩玩 abc", 5), substr("abc 玩玩玩 abc", 5, 3) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q index f103da9ecb2de..3980fe83fbcb2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q @@ -1,16 +1,18 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION tan; DESCRIBE FUNCTION EXTENDED tan; SELECT tan(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT tan(1), tan(6), tan(-1.0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); DESCRIBE FUNCTION tan; DESCRIBE FUNCTION EXTENDED tan; SELECT tan(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT tan(1), tan(6), tan(-1.0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q index 322a061d645d7..c94a52133d324 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q @@ -1,12 +1,10 @@ +set hive.fetch.task.conversion=more; + EXPLAIN CREATE TEMPORARY FUNCTION testlength AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength'; CREATE TEMPORARY FUNCTION testlength AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength'; -CREATE TABLE dest1(len INT); - -FROM src INSERT OVERWRITE TABLE dest1 SELECT testlength(src.value); - -SELECT dest1.* FROM dest1; +SELECT testlength(src.value) FROM src; DROP TEMPORARY FUNCTION testlength; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q index 6de270902f7fb..27e46c24a83d5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q @@ -1,12 +1,10 @@ +set hive.fetch.task.conversion=more; + EXPLAIN CREATE TEMPORARY FUNCTION testlength2 AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength2'; CREATE TEMPORARY FUNCTION testlength2 AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength2'; -CREATE TABLE dest1(len INT); - -FROM src INSERT OVERWRITE TABLE dest1 SELECT testlength2(src.value); - -SELECT dest1.* FROM dest1; +SELECT testlength2(src.value) FROM src; DROP TEMPORARY FUNCTION testlength2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q index ca23f719f93e3..8bea7abcbc4d3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q @@ -1,44 +1,46 @@ --- 'true' cases: +set hive.fetch.task.conversion=more; -SELECT CAST(CAST(1 AS TINYINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(2 AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(-4 AS INT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(-444 AS BIGINT) AS BOOLEAN) FROM src LIMIT 1; +-- 'true' cases: -SELECT CAST(CAST(7.0 AS FLOAT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(-8.0 AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(-99.0 AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(1 AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(2 AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(-4 AS INT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(-444 AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST('Foo' AS STRING) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(7.0 AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(-8.0 AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(-99.0 AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST('2011-05-06 07:08:09' as timestamp) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST('Foo' AS STRING) AS BOOLEAN) FROM src tablesample (1 rows); + +SELECT CAST(CAST('2011-05-06 07:08:09' as timestamp) AS BOOLEAN) FROM src tablesample (1 rows); -- 'false' cases: -SELECT CAST(CAST(0 AS TINYINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0 AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0 AS INT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0 AS BIGINT) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(0 AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0 AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0 AS INT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0 AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(0.0 AS FLOAT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0.0 AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0.0 AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(0.0 AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0.0 AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0.0 AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST('' AS STRING) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST('' AS STRING) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(0 as timestamp) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(0 as timestamp) AS BOOLEAN) FROM src tablesample (1 rows); -- 'NULL' cases: -SELECT CAST(NULL AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(NULL AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(NULL AS TINYINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS INT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS BIGINT) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(NULL AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS INT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(NULL AS FLOAT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(NULL AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(NULL AS STRING) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL as timestamp) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(NULL AS STRING) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL as timestamp) AS BOOLEAN) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q index ded930d1115a4..aa0a250e4678d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Byte type: -SELECT CAST(NULL AS TINYINT) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS TINYINT) FROM src LIMIT 1; +-- Conversion of main primitive types to Byte type: +SELECT CAST(NULL AS TINYINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-18 AS SMALLINT) AS TINYINT) FROM src LIMIT 1; -SELECT CAST(-129 AS TINYINT) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS TINYINT) FROM src LIMIT 1; +SELECT CAST(TRUE AS TINYINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS TINYINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS FLOAT) AS TINYINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS TINYINT) FROM src LIMIT 1; +SELECT CAST(CAST(-18 AS SMALLINT) AS TINYINT) FROM src tablesample (1 rows); +SELECT CAST(-129 AS TINYINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS TINYINT) FROM src tablesample (1 rows); -SELECT CAST('-38' AS TINYINT) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS TINYINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS FLOAT) AS TINYINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL) AS TINYINT) FROM src tablesample (1 rows); + +SELECT CAST('-38' AS TINYINT) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q index b0a248ad70bba..005ec9d24e1ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Double type: -SELECT CAST(NULL AS DOUBLE) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS DOUBLE) FROM src LIMIT 1; +-- Conversion of main primitive types to Double type: +SELECT CAST(NULL AS DOUBLE) FROM src tablesample (1 rows); -SELECT CAST(CAST(-7 AS TINYINT) AS DOUBLE) FROM src LIMIT 1; -SELECT CAST(CAST(-18 AS SMALLINT) AS DOUBLE) FROM src LIMIT 1; -SELECT CAST(-129 AS DOUBLE) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS DOUBLE) FROM src LIMIT 1; +SELECT CAST(TRUE AS DOUBLE) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS FLOAT) AS DOUBLE) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS DOUBLE) FROM src LIMIT 1; +SELECT CAST(CAST(-7 AS TINYINT) AS DOUBLE) FROM src tablesample (1 rows); +SELECT CAST(CAST(-18 AS SMALLINT) AS DOUBLE) FROM src tablesample (1 rows); +SELECT CAST(-129 AS DOUBLE) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS DOUBLE) FROM src tablesample (1 rows); -SELECT CAST('-38.14' AS DOUBLE) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS FLOAT) AS DOUBLE) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS DOUBLE) FROM src tablesample (1 rows); + +SELECT CAST('-38.14' AS DOUBLE) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q index c91d18cc2f5c7..95671f15fe0ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Float type: -SELECT CAST(NULL AS FLOAT) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS FLOAT) FROM src LIMIT 1; +-- Conversion of main primitive types to Float type: +SELECT CAST(NULL AS FLOAT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-7 AS TINYINT) AS FLOAT) FROM src LIMIT 1; -SELECT CAST(CAST(-18 AS SMALLINT) AS FLOAT) FROM src LIMIT 1; -SELECT CAST(-129 AS FLOAT) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS FLOAT) FROM src LIMIT 1; +SELECT CAST(TRUE AS FLOAT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS FLOAT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS FLOAT) FROM src LIMIT 1; +SELECT CAST(CAST(-7 AS TINYINT) AS FLOAT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-18 AS SMALLINT) AS FLOAT) FROM src tablesample (1 rows); +SELECT CAST(-129 AS FLOAT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS FLOAT) FROM src tablesample (1 rows); -SELECT CAST('-38.14' AS FLOAT) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS FLOAT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS FLOAT) FROM src tablesample (1 rows); + +SELECT CAST('-38.14' AS FLOAT) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q index 45dc6f8bd32fa..706411a398463 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Long type: -SELECT CAST(NULL AS BIGINT) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS BIGINT) FROM src LIMIT 1; +-- Conversion of main primitive types to Long type: +SELECT CAST(NULL AS BIGINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-7 AS TINYINT) AS BIGINT) FROM src LIMIT 1; -SELECT CAST(CAST(-18 AS SMALLINT) AS BIGINT) FROM src LIMIT 1; -SELECT CAST(-129 AS BIGINT) FROM src LIMIT 1; +SELECT CAST(TRUE AS BIGINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS BIGINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS FLOAT) AS BIGINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS BIGINT) FROM src LIMIT 1; +SELECT CAST(CAST(-7 AS TINYINT) AS BIGINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-18 AS SMALLINT) AS BIGINT) FROM src tablesample (1 rows); +SELECT CAST(-129 AS BIGINT) FROM src tablesample (1 rows); -SELECT CAST('-38' AS BIGINT) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS BIGINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS FLOAT) AS BIGINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL) AS BIGINT) FROM src tablesample (1 rows); + +SELECT CAST('-38' AS BIGINT) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q index 7d843c1ea12ea..5cc4e57c8c9b5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Short type: -SELECT CAST(NULL AS SMALLINT) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS SMALLINT) FROM src LIMIT 1; +-- Conversion of main primitive types to Short type: +SELECT CAST(NULL AS SMALLINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-18 AS TINYINT) AS SMALLINT) FROM src LIMIT 1; -SELECT CAST(-129 AS SMALLINT) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS SMALLINT) FROM src LIMIT 1; +SELECT CAST(TRUE AS SMALLINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS SMALLINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS FLOAT) AS SMALLINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS SMALLINT) FROM src LIMIT 1; +SELECT CAST(CAST(-18 AS TINYINT) AS SMALLINT) FROM src tablesample (1 rows); +SELECT CAST(-129 AS SMALLINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS SMALLINT) FROM src tablesample (1 rows); -SELECT CAST('-38' AS SMALLINT) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS SMALLINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS FLOAT) AS SMALLINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL) AS SMALLINT) FROM src tablesample (1 rows); + +SELECT CAST('-38' AS SMALLINT) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q index 3b585e7170c60..ac4b5242e1fc4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q @@ -1,16 +1,18 @@ --- Conversion of main primitive types to String type: -SELECT CAST(NULL AS STRING) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS STRING) FROM src LIMIT 1; +-- Conversion of main primitive types to String type: +SELECT CAST(NULL AS STRING) FROM src tablesample (1 rows); -SELECT CAST(CAST(1 AS TINYINT) AS STRING) FROM src LIMIT 1; -SELECT CAST(CAST(-18 AS SMALLINT) AS STRING) FROM src LIMIT 1; -SELECT CAST(-129 AS STRING) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS STRING) FROM src LIMIT 1; +SELECT CAST(TRUE AS STRING) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS STRING) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS FLOAT) AS STRING) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS STRING) FROM src LIMIT 1; +SELECT CAST(CAST(1 AS TINYINT) AS STRING) FROM src tablesample (1 rows); +SELECT CAST(CAST(-18 AS SMALLINT) AS STRING) FROM src tablesample (1 rows); +SELECT CAST(-129 AS STRING) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS STRING) FROM src tablesample (1 rows); -SELECT CAST('Foo' AS STRING) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS STRING) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS FLOAT) AS STRING) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS STRING) FROM src tablesample (1 rows); + +SELECT CAST('Foo' AS STRING) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q index 3024074bba4b3..0a2758edfcd4f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION to_unix_timestamp; DESCRIBE FUNCTION EXTENDED to_unix_timestamp; create table oneline(key int, value string); -load data local inpath '../data/files/things.txt' into table oneline; +load data local inpath '../../data/files/things.txt' into table oneline; SELECT '2009-03-20 11:30:01', diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q index cba6ff90f643f..21d799882f59b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION translate; DESCRIBE FUNCTION EXTENDED translate; @@ -10,28 +12,28 @@ FROM src INSERT OVERWRITE TABLE table_translate SELECT 'abcd', 'ahd', '12' WHERE -- Run some queries on constant input parameters SELECT translate('abcd', 'ab', '12'), - translate('abcd', 'abc', '12') FROM src LIMIT 1; + translate('abcd', 'abc', '12') FROM src tablesample (1 rows); -- Run some queries where first parameter being a table column while the other two being constants SELECT translate(table_input.input, 'ab', '12'), - translate(table_input.input, 'abc', '12') FROM table_input LIMIT 1; + translate(table_input.input, 'abc', '12') FROM table_input tablesample (1 rows); -- Run some queries where all parameters are coming from table columns -SELECT translate(input_string, from_string, to_string) FROM table_translate LIMIT 1; +SELECT translate(input_string, from_string, to_string) FROM table_translate tablesample (1 rows); -- Run some queries where some parameters are NULL SELECT translate(NULL, 'ab', '12'), translate('abcd', NULL, '12'), translate('abcd', 'ab', NULL), - translate(NULL, NULL, NULL) FROM src LIMIT 1; + translate(NULL, NULL, NULL) FROM src tablesample (1 rows); -- Run some queries where the same character appears several times in the from string (2nd argument) of the UDF SELECT translate('abcd', 'aba', '123'), - translate('abcd', 'aba', '12') FROM src LIMIT 1; + translate('abcd', 'aba', '12') FROM src tablesample (1 rows); -- Run some queries for the ignorant case when the 3rd parameter has more characters than the second one -SELECT translate('abcd', 'abc', '1234') FROM src LIMIT 1; +SELECT translate('abcd', 'abc', '1234') FROM src tablesample (1 rows); -- Test proper function over UTF-8 characters -SELECT translate('Àbcd', 'À', 'Ã') FROM src LIMIT 1; +SELECT translate('Àbcd', 'À', 'Ã') FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q index e80021c8b2c51..257e469ffb16c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION unhex; DESCRIBE FUNCTION EXTENDED unhex; @@ -9,11 +11,11 @@ SELECT unhex('61'), unhex('2D34'), unhex('') -FROM src limit 1; +FROM src tablesample (1 rows); -- Bad inputs SELECT unhex('MySQL'), unhex('G123'), unhex('\0') -FROM src limit 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q index 9140d22fb58c7..3876beb17d7ce 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION create_union; DESCRIBE FUNCTION EXTENDED create_union; EXPLAIN SELECT create_union(0, key), create_union(if(key<100, 0, 1), 2.0, value), create_union(1, "a", struct(2, "b")) -FROM src LIMIT 2; +FROM src tablesample (2 rows); SELECT create_union(0, key), create_union(if(key<100, 0, 1), 2.0, value), create_union(1, "a", struct(2, "b")) -FROM src LIMIT 2; +FROM src tablesample (2 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q index 89288a1193c87..1664329c33d98 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION unix_timestamp; DESCRIBE FUNCTION EXTENDED unix_timestamp; create table oneline(key int, value string); -load data local inpath '../data/files/things.txt' into table oneline; +load data local inpath '../../data/files/things.txt' into table oneline; SELECT '2009-03-20 11:30:01', diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q new file mode 100644 index 0000000000000..093187ddc30cb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q @@ -0,0 +1,15 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/udf_using; + +dfs -copyFromLocal ../../data/files/sales.txt hdfs:///tmp/udf_using/sales.txt; + +create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file 'hdfs:///tmp/udf_using/sales.txt'; + +create table udf_using (c1 string); +insert overwrite table udf_using select 'Joe' from src limit 2; + +select c1, lookup(c1) from udf_using; + +drop table udf_using; +drop function lookup; + +dfs -rmr hdfs:///tmp/udf_using; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q index 4b7b4ea55aa81..abb0a2d7d2c08 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q @@ -1,6 +1,8 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION weekofyear; DESCRIBE FUNCTION EXTENDED weekofyear; SELECT weekofyear('1980-01-01'), weekofyear('1980-01-06'), weekofyear('1980-01-07'), weekofyear('1980-12-31'), weekofyear('1984-1-1'), weekofyear('2008-02-20 00:00:00'), weekofyear('1980-12-28 23:59:59'), weekofyear('1980-12-29 23:59:59') -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q index d94a594f026c5..4eb7f6918a0dc 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION when; DESCRIBE FUNCTION EXTENDED when; @@ -27,7 +29,7 @@ SELECT CASE WHEN 25=26 THEN 27 WHEN 28=28 THEN NULL END -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT CASE WHEN 1=1 THEN 2 @@ -54,4 +56,21 @@ SELECT CASE WHEN 25=26 THEN 27 WHEN 28=28 THEN NULL END -FROM src LIMIT 1; +FROM src tablesample (1 rows); + +-- Allow compatible types to be used in return value +SELECT CASE + WHEN 1=1 THEN 123.0BD + ELSE 0.0BD + END, + CASE + WHEN 1=1 THEN 123 + WHEN 1=2 THEN 1.0 + ELSE 222.02BD + END, + CASE + WHEN 1=1 THEN 'abcd' + WHEN 1=2 THEN cast('efgh' as varchar(10)) + ELSE cast('ijkl' as char(4)) + END +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q index fca1ba11d85b2..1ad38abcf6ec5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath ; DESCRIBE FUNCTION EXTENDED xpath ; -SELECT xpath ('b1b2b3c1c2', 'a/text()') FROM src LIMIT 1 ; -SELECT xpath ('b1b2b3c1c2', 'a/*/text()') FROM src LIMIT 1 ; -SELECT xpath ('b1b2b3c1c2', 'a/b/text()') FROM src LIMIT 1 ; -SELECT xpath ('b1b2b3c1c2', 'a/c/text()') FROM src LIMIT 1 ; -SELECT xpath ('b1b2b3c1c2', 'a/*[@class="bb"]/text()') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath ('b1b2b3c1c2', 'a/text()') FROM src tablesample (1 rows) ; +SELECT xpath ('b1b2b3c1c2', 'a/*/text()') FROM src tablesample (1 rows) ; +SELECT xpath ('b1b2b3c1c2', 'a/b/text()') FROM src tablesample (1 rows) ; +SELECT xpath ('b1b2b3c1c2', 'a/c/text()') FROM src tablesample (1 rows) ; +SELECT xpath ('b1b2b3c1c2', 'a/*[@class="bb"]/text()') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q index 3a6e613eb6a0e..6e3ff244b0434 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q @@ -1,9 +1,11 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_boolean ; DESCRIBE FUNCTION EXTENDED xpath_boolean ; -SELECT xpath_boolean ('b', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_boolean ('b', 'a/c') FROM src LIMIT 1 ; -SELECT xpath_boolean ('b', 'a/b = "b"') FROM src LIMIT 1 ; -SELECT xpath_boolean ('b', 'a/b = "c"') FROM src LIMIT 1 ; -SELECT xpath_boolean ('10', 'a/b < 10') FROM src LIMIT 1 ; -SELECT xpath_boolean ('10', 'a/b = 10') FROM src LIMIT 1 ; +SELECT xpath_boolean ('b', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('b', 'a/c') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('b', 'a/b = "b"') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('b', 'a/b = "c"') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('10', 'a/b < 10') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('10', 'a/b = 10') FROM src tablesample (1 rows) ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q index 4328747f44497..68441762e5a66 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q @@ -1,14 +1,16 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_number ; DESCRIBE FUNCTION EXTENDED xpath_number ; DESCRIBE FUNCTION xpath_double ; DESCRIBE FUNCTION EXTENDED xpath_double ; -SELECT xpath_double ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_double ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_double ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_double ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_double ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_double ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_double ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_double ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath_double ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_double ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_double ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_double ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_double ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_double ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_double ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_double ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q index 1f1482833c858..4596a322604a5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_float ; DESCRIBE FUNCTION EXTENDED xpath_float ; -SELECT xpath_float ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_float ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_float ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_float ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_float ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_float ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_float ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_float ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath_float ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_float ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_float ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_float ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_float ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_float ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_float ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_float ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q index 9b50bd9eeb365..9f3898f1147ac 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_int ; DESCRIBE FUNCTION EXTENDED xpath_int ; -SELECT xpath_int ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_int ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_int ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_int ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_int ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_int ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_int ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_int ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath_int ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_int ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_int ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_int ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_int ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_int ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_int ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_int ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q index 04ee61b1f0e9d..3a335937c54fb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_long ; DESCRIBE FUNCTION EXTENDED xpath_long ; -SELECT xpath_long ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_long ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_long ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_long ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_long ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_long ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_long ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_long ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; +SELECT xpath_long ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_long ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_long ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_long ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_long ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_long ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_long ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_long ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q index 6a1abdc0bdb21..073056e72e7a1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_short ; DESCRIBE FUNCTION EXTENDED xpath_short ; -SELECT xpath_short ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_short ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_short ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_short ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_short ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_short ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_short ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_short ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath_short ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_short ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_short ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_short ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_short ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_short ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_short ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_short ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q index ebbc913b71bfb..1f1731c67d114 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_string ; DESCRIBE FUNCTION EXTENDED xpath_string ; -SELECT xpath_string ('bbcc', 'a') FROM src LIMIT 1 ; -SELECT xpath_string ('bbcc', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_string ('bbcc', 'a/c') FROM src LIMIT 1 ; -SELECT xpath_string ('bbcc', 'a/d') FROM src LIMIT 1 ; -SELECT xpath_string ('b1b2', '//b') FROM src LIMIT 1 ; -SELECT xpath_string ('b1b2', 'a/b[1]') FROM src LIMIT 1 ; -SELECT xpath_string ('b1b2', 'a/b[2]') FROM src LIMIT 1 ; -SELECT xpath_string ('b1b2', 'a/b[@id="b_2"]') FROM src LIMIT 1 ; +SELECT xpath_string ('bbcc', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_string ('bbcc', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_string ('bbcc', 'a/c') FROM src tablesample (1 rows) ; +SELECT xpath_string ('bbcc', 'a/d') FROM src tablesample (1 rows) ; +SELECT xpath_string ('b1b2', '//b') FROM src tablesample (1 rows) ; +SELECT xpath_string ('b1b2', 'a/b[1]') FROM src tablesample (1 rows) ; +SELECT xpath_string ('b1b2', 'a/b[2]') FROM src tablesample (1 rows) ; +SELECT xpath_string ('b1b2', 'a/b[@id="b_2"]') FROM src tablesample (1 rows) ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q index 638a4e9ca57e7..1d405b3560c5c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION explode; DESCRIBE FUNCTION EXTENDED explode; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q index 712d9598c755d..93d829d4ed175 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q @@ -2,17 +2,17 @@ create table json_t (key string, jstring string); insert overwrite table json_t select * from ( - select '1', '{"f1": "value1", "f2": "value2", "f3": 3, "f5": 5.23}' from src limit 1 + select '1', '{"f1": "value1", "f2": "value2", "f3": 3, "f5": 5.23}' from src tablesample (1 rows) union all - select '2', '{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}' from src limit 1 + select '2', '{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}' from src tablesample (1 rows) union all - select '3', '{"f1": "value13", "f4": "value44", "f3": "value33", "f2": 2, "f5": 5.01}' from src limit 1 + select '3', '{"f1": "value13", "f4": "value44", "f3": "value33", "f2": 2, "f5": 5.01}' from src tablesample (1 rows) union all - select '4', cast(null as string) from src limit 1 + select '4', cast(null as string) from src tablesample (1 rows) union all - select '5', '{"f1": "", "f5": null}' from src limit 1 + select '5', '{"f1": "", "f5": null}' from src tablesample (1 rows) union all - select '6', '[invalid JSON string]' from src limit 1 + select '6', '[invalid JSON string]' from src tablesample (1 rows) ) s; explain @@ -40,7 +40,7 @@ select f2, count(*) from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2', CREATE TABLE dest1(c1 STRING) STORED AS RCFILE; -INSERT OVERWRITE TABLE dest1 SELECT '{"a":"b\nc"}' FROM src LIMIT 1; +INSERT OVERWRITE TABLE dest1 SELECT '{"a":"b\nc"}' FROM src tablesample (1 rows); SELECT * FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q index 055e39b2973a6..0870cbc4a9950 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q @@ -2,17 +2,17 @@ create table url_t (key string, fullurl string); insert overwrite table url_t select * from ( - select '1', 'http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1' from src limit 1 + select '1', 'http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1' from src tablesample (1 rows) union all - select '2', 'https://www.socs.uts.edu.au:80/MosaicDocs-old/url-primer.html?k1=tps#chapter1' from src limit 1 + select '2', 'https://www.socs.uts.edu.au:80/MosaicDocs-old/url-primer.html?k1=tps#chapter1' from src tablesample (1 rows) union all - select '3', 'ftp://sites.google.com/a/example.com/site/page' from src limit 1 + select '3', 'ftp://sites.google.com/a/example.com/site/page' from src tablesample (1 rows) union all - select '4', cast(null as string) from src limit 1 + select '4', cast(null as string) from src tablesample (1 rows) union all - select '5', 'htttp://' from src limit 1 + select '5', 'htttp://' from src tablesample (1 rows) union all - select '6', '[invalid url string]' from src limit 1 + select '6', '[invalid url string]' from src tablesample (1 rows) ) s; describe function parse_url_tuple; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q new file mode 100644 index 0000000000000..343f08ba6fe4b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q @@ -0,0 +1,15 @@ +CREATE TABLE employees ( +name STRING, +salary FLOAT, +subordinates ARRAY, +deductions MAP, +address STRUCT); + +LOAD DATA LOCAL INPATH '../../data/files/posexplode_data.txt' INTO TABLE employees; + +SELECT + name, pos, sub +FROM + employees +LATERAL VIEW + posexplode(subordinates) subView AS pos, sub; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q index 91bbd1b07f08b..525eccbbfe227 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q @@ -6,13 +6,13 @@ FROM ( UNION ALL FROM src SELECT src.* WHERE src.key > 100 ) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*; +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; 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.*; +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; -dfs -cat ../build/ql/test/data/warehouse/union.out/*; +dfs -cat ${system:test.warehouse.dir}/union.out/*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q index 36bc865be81fe..238b583fd86a8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q @@ -3,7 +3,7 @@ create table src10_2 (key string, value string); create table src10_3 (key string, value string); create table src10_4 (key string, value string); -from (select * from src limit 10) a +from (select * from src tablesample (10 rows)) a insert overwrite table src10_1 select * insert overwrite table src10_2 select * insert overwrite table src10_3 select * diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q index e332a8af7bb1b..dd6f08e5b4e21 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q @@ -17,8 +17,8 @@ create table union_date_2 ( FL_NUM int ); -LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_1; -LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_2; +LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_1; +LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_2; select * from ( select fl_num, fl_date from union_date_1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q index 70147070feaca..4368b8a5b6c36 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q @@ -1,2 +1,5 @@ -- HIVE-2901 select x from (select value as x from src union all select NULL as x from src)a limit 10; + +-- HIVE-4837 +select * from (select null as N from src1 group by key UNION ALL select null as N from src1 group by key ) a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q index c9f920c648a92..c87b3fef1e462 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q @@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q index a072fe3b56e43..6701952717eb5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q index 6250a20210f9f..4b2fa42f1b54d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q index 168eac34a0301..69d0d0af9ff68 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q index a88a83e152a5f..7605f0ec26a08 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q index e588e8fd62ab9..a4fdfc8ee44ef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q @@ -25,7 +25,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q index 237f2e7629057..e3c937b7b1e06 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q @@ -25,7 +25,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 partition (ds) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q index 06d5043f9cf9d..537078b080433 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q @@ -25,7 +25,7 @@ set hive.exec.dynamic.partition=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile ; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 partition (ds) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q index 65b8255c3e954..d70f3d3da31a2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q @@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 partition (ds) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q index 265acfd35758f..478650038cc9c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, ds string) stored as textfile; create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 partition (ds) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q index 1450caa2d07a7..8c45953ed877c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q @@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q index 015c146ecadf0..83cd2887ab23f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q index ac727889273d7..f80f7c1dfdadd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(values bigint, key string) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q index f89744624116f..8963c2576ca76 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q index f01053d27d0c0..b0c1ccd73db68 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q @@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint, values2 bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q index 805dd76274c42..a1b989a0eb63c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q index 36fd947d6035d..ec561e0979e0c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q @@ -18,7 +18,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key double, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; EXPLAIN INSERT OVERWRITE TABLE outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q index da0f1c0473ffe..9617f737bc552 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q index 18d4730a12024..cae323b1ef89d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q @@ -20,7 +20,7 @@ set hive.merge.smallfiles.avgsize=1; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q index a6fbeb03ddcc3..5df84e145c709 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q @@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q index 7ae5af30c1bae..bfce26d0fbe1f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q @@ -17,7 +17,7 @@ create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; create table outputTbl2(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain FROM ( diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q index 5a639ca11700d..3a956747a473d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q @@ -21,7 +21,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q index 79b84e81181d3..a83a43e466f55 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q @@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q index f6038384f3fa8..e71f6dd001dd6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q @@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q new file mode 100644 index 0000000000000..6a4b45fbc4599 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q @@ -0,0 +1,106 @@ +-- top level +explain +select * from ( +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3 +) q1 order by key; + +select * from ( +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3 +) q1 order by key; + +explain +select * from ( +select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10 +union all +select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10 +) q1 order by k; + +select * from ( +select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10 +union all +select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10 +) q1 order by k; + +-- ctas +explain +create table union_top as +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +create table union_top as +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +select * from union_top order by key; + +truncate table union_top; + +-- insert into +explain +insert into table union_top +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +insert into table union_top +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +select * from union_top order by key; + +explain +insert overwrite table union_top +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +insert overwrite table union_top +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +select * from union_top order by key; + +-- create view +explain +create view union_top_view as +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +create view union_top_view as +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +select * from union_top_view order by key; + +drop table union_top; +drop view union_top_view; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q index 4f8bafe37a809..b727199cf4f25 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string); CREATE INDEX src_union_1_key_idx ON TABLE src_union_1(key) AS 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q index 51bcf22bfa0be..3bc8ef931bcd3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q @@ -2,9 +2,9 @@ CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; FROM UNIQUEJOIN PRESERVE T1 a (a.key), PRESERVE T2 b (b.key), PRESERVE T3 c (c.key) SELECT a.key, b.key, c.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q index 53273b3187a78..bed61f381ce96 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q @@ -5,7 +5,7 @@ create table varchar1 (key varchar(10), value varchar(20)); create table varchar1_1 (key string, value string); -- load from file -load data local inpath '../data/files/srcbucket0.txt' overwrite into table varchar1; +load data local inpath '../../data/files/srcbucket0.txt' overwrite into table varchar1; select * from varchar1 order by key, value limit 2; -- insert overwrite, from same/different length varchar diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q index 550f3dc728df2..c356b1dbca6ab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; -- Cast from varchar to other data types select diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q index b6c6f406412fe..05cad852a2680 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; -- Should all be true select diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q index 6a19efaa3c5a1..94226879c53ab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q @@ -17,9 +17,9 @@ create table varchar_join1_str ( c2 string ); -load data local inpath '../data/files/vc1.txt' into table varchar_join1_vc1; -load data local inpath '../data/files/vc1.txt' into table varchar_join1_vc2; -load data local inpath '../data/files/vc1.txt' into table varchar_join1_str; +load data local inpath '../../data/files/vc1.txt' into table varchar_join1_vc1; +load data local inpath '../../data/files/vc1.txt' into table varchar_join1_vc2; +load data local inpath '../../data/files/vc1.txt' into table varchar_join1_str; -- Join varchar with same length varchar select * from varchar_join1_vc1 a join varchar_join1_vc1 b on (a.c2 = b.c2) order by a.c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q index 7351b688cf504..ea2a022b94037 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q @@ -18,7 +18,7 @@ with serdeproperties ( ) stored as textfile; -load data local inpath '../data/files/srcbucket0.txt' overwrite into table varchar_serde_regex; +load data local inpath '../../data/files/srcbucket0.txt' overwrite into table varchar_serde_regex; select * from varchar_serde_regex limit 5; select value, count(*) from varchar_serde_regex group by value limit 5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q index cf90eab33e2d0..dd3cffe0db4fe 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q @@ -17,9 +17,9 @@ create table varchar_union1_str ( c2 string ); -load data local inpath '../data/files/vc1.txt' into table varchar_union1_vc1; -load data local inpath '../data/files/vc1.txt' into table varchar_union1_vc2; -load data local inpath '../data/files/vc1.txt' into table varchar_union1_str; +load data local inpath '../../data/files/vc1.txt' into table varchar_union1_vc1; +load data local inpath '../../data/files/vc1.txt' into table varchar_union1_vc2; +load data local inpath '../../data/files/vc1.txt' into table varchar_union1_str; -- union varchar with same length varchar select * from ( diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q new file mode 100644 index 0000000000000..1bc66118f8ff3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q @@ -0,0 +1,35 @@ +SET hive.vectorized.execution.enabled=true; + +CREATE TABLE decimal_date_test STORED AS ORC AS SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, CAST(CAST((CAST(cint AS BIGINT) *ctinyint) AS TIMESTAMP) AS DATE) AS cdate FROM alltypesorc ORDER BY cdate; + +EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE)) ORDER BY cdate; + +EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdate NOT IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE), CAST("1970-01-21" AS DATE)); + +EXPLAIN SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 IN (2365.8945945946, 881.0135135135, -3367.6517567568) ORDER BY cdecimal1; + +EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT IN (2365.8945945946, 881.0135135135, -3367.6517567568); + +EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate BETWEEN CAST("1969-12-30" AS DATE) AND CAST("1970-01-02" AS DATE) ORDER BY cdate; + +EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate NOT BETWEEN CAST("1968-05-01" AS DATE) AND CAST("1971-09-01" AS DATE) ORDER BY cdate; + +EXPLAIN SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 BETWEEN -20 AND 45.9918918919 ORDER BY cdecimal1; + +EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT BETWEEN -2000 AND 4390.1351351351; + +SELECT cdate FROM decimal_date_test WHERE cdate IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE)) ORDER BY cdate; + +SELECT COUNT(*) FROM decimal_date_test WHERE cdate NOT IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE), CAST("1970-01-21" AS DATE)); + +SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 IN (2365.8945945946, 881.0135135135, -3367.6517567568) ORDER BY cdecimal1; + +SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT IN (2365.8945945946, 881.0135135135, -3367.6517567568); + +SELECT cdate FROM decimal_date_test WHERE cdate BETWEEN CAST("1969-12-30" AS DATE) AND CAST("1970-01-02" AS DATE) ORDER BY cdate; + +SELECT cdate FROM decimal_date_test WHERE cdate NOT BETWEEN CAST("1968-05-01" AS DATE) AND CAST("1971-09-01" AS DATE) ORDER BY cdate; + +SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 BETWEEN -20 AND 45.9918918919 ORDER BY cdecimal1; + +SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT BETWEEN -2000 AND 4390.1351351351; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q new file mode 100644 index 0000000000000..052ab716bee1a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q @@ -0,0 +1,32 @@ +SET hive.vectorized.execution.enabled=true; +EXPLAIN SELECT cdouble, cstring1, cint, cfloat, csmallint, coalesce(cdouble, cstring1, cint, cfloat, csmallint) +FROM alltypesorc +WHERE (cdouble IS NULL) LIMIT 10; + +SELECT cdouble, cstring1, cint, cfloat, csmallint, coalesce(cdouble, cstring1, cint, cfloat, csmallint) +FROM alltypesorc +WHERE (cdouble IS NULL) LIMIT 10; + +EXPLAIN SELECT ctinyint, cdouble, cint, coalesce(ctinyint+10, (cdouble+log2(cint)), 0) +FROM alltypesorc +WHERE (ctinyint IS NULL) LIMIT 10; + +SELECT ctinyint, cdouble, cint, coalesce(ctinyint+10, (cdouble+log2(cint)), 0) +FROM alltypesorc +WHERE (ctinyint IS NULL) LIMIT 10; + +EXPLAIN SELECT cfloat, cbigint, coalesce(cfloat, cbigint, 0) +FROM alltypesorc +WHERE (cfloat IS NULL AND cbigint IS NULL) LIMIT 10; + +SELECT cfloat, cbigint, coalesce(cfloat, cbigint, 0) +FROM alltypesorc +WHERE (cfloat IS NULL AND cbigint IS NULL) LIMIT 10; + +EXPLAIN SELECT ctimestamp1, ctimestamp2, coalesce(ctimestamp1, ctimestamp2) +FROM alltypesorc +WHERE ctimestamp1 IS NOT NULL OR ctimestamp2 IS NOT NULL LIMIT 10; + +SELECT ctimestamp1, ctimestamp2, coalesce(ctimestamp1, ctimestamp2) +FROM alltypesorc +WHERE ctimestamp1 IS NOT NULL OR ctimestamp2 IS NOT NULL LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q new file mode 100644 index 0000000000000..eb9146e95d437 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q @@ -0,0 +1,20 @@ +CREATE TABLE decimal_vgby STORED AS ORC AS + SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, + CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, + cint + FROM alltypesorc; + +SET hive.vectorized.execution.enabled=true; + +EXPLAIN SELECT cint, + COUNT(cdecimal1), MAX(cdecimal1), MIN(cdecimal1), SUM(cdecimal1), AVG(cdecimal1), STDDEV_POP(cdecimal1), STDDEV_SAMP(cdecimal1), + COUNT(cdecimal2), MAX(cdecimal2), MIN(cdecimal2), SUM(cdecimal2), AVG(cdecimal2), STDDEV_POP(cdecimal2), STDDEV_SAMP(cdecimal2) + FROM decimal_vgby + GROUP BY cint + HAVING COUNT(*) > 1; +SELECT cint, + COUNT(cdecimal1), MAX(cdecimal1), MIN(cdecimal1), SUM(cdecimal1), AVG(cdecimal1), STDDEV_POP(cdecimal1), STDDEV_SAMP(cdecimal1), + COUNT(cdecimal2), MAX(cdecimal2), MIN(cdecimal2), SUM(cdecimal2), AVG(cdecimal2), STDDEV_POP(cdecimal2), STDDEV_SAMP(cdecimal2) + FROM decimal_vgby + GROUP BY cint + HAVING COUNT(*) > 1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q new file mode 100644 index 0000000000000..ea7a5b817d7c0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q @@ -0,0 +1,5 @@ +SET hive.vectorized.execution.enabled=true; + +EXPLAIN SELECT cdouble, cint, cboolean1, ctimestamp1, CAST(cdouble AS DECIMAL(20,10)), CAST(cint AS DECIMAL(23,14)), CAST(cboolean1 AS DECIMAL(5,2)), CAST(ctimestamp1 AS DECIMAL(15,0)) FROM alltypesorc WHERE cdouble IS NOT NULL AND cint IS NOT NULL AND cboolean1 IS NOT NULL AND ctimestamp1 IS NOT NULL LIMIT 10; + +SELECT cdouble, cint, cboolean1, ctimestamp1, CAST(cdouble AS DECIMAL(20,10)), CAST(cint AS DECIMAL(23,14)), CAST(cboolean1 AS DECIMAL(5,2)), CAST(ctimestamp1 AS DECIMAL(15,0)) FROM alltypesorc WHERE cdouble IS NOT NULL AND cint IS NOT NULL AND cboolean1 IS NOT NULL AND ctimestamp1 IS NOT NULL LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q new file mode 100644 index 0000000000000..a74b17bd127d6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q @@ -0,0 +1,5 @@ +CREATE TABLE decimal_test STORED AS ORC AS SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2 FROM alltypesorc; +SET hive.vectorized.execution.enabled=true; +EXPLAIN SELECT cdecimal1 + cdecimal2, cdecimal1 - (2*cdecimal2), ((cdecimal1+2.34)/cdecimal2), (cdecimal1 * (cdecimal2/3.4)), cdecimal1 % 10, CAST(cdecimal1 AS INT), CAST(cdecimal2 AS SMALLINT), CAST(cdecimal2 AS TINYINT), CAST(cdecimal1 AS BIGINT), CAST (cdecimal1 AS BOOLEAN), CAST(cdecimal2 AS DOUBLE), CAST(cdecimal1 AS FLOAT), CAST(cdecimal2 AS STRING), CAST(cdecimal1 AS TIMESTAMP) FROM decimal_test WHERE cdecimal1 > 0 AND cdecimal1 < 12345.5678 AND cdecimal2 != 0 AND cdecimal2 > 1000 AND cdouble IS NOT NULL LIMIT 10; + +SELECT cdecimal1 + cdecimal2, cdecimal1 - (2*cdecimal2), ((cdecimal1+2.34)/cdecimal2), (cdecimal1 * (cdecimal2/3.4)), cdecimal1 % 10, CAST(cdecimal1 AS INT), CAST(cdecimal2 AS SMALLINT), CAST(cdecimal2 AS TINYINT), CAST(cdecimal1 AS BIGINT), CAST (cdecimal1 AS BOOLEAN), CAST(cdecimal2 AS DOUBLE), CAST(cdecimal1 AS FLOAT), CAST(cdecimal2 AS STRING), CAST(cdecimal1 AS TIMESTAMP) FROM decimal_test WHERE cdecimal1 > 0 AND cdecimal1 < 12345.5678 AND cdecimal2 != 0 AND cdecimal2 > 1000 AND cdouble IS NOT NULL LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q new file mode 100644 index 0000000000000..d8b3d1a9ac13f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q @@ -0,0 +1,19 @@ +CREATE TABLE decimal_mapjoin STORED AS ORC AS + SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, + CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, + cint + FROM alltypesorc; + +SET hive.auto.convert.join=true; +SET hive.auto.convert.join.nonconditionaltask=true; +SET hive.auto.convert.join.nonconditionaltask.size=1000000000; +SET hive.vectorized.execution.enabled=true; + +EXPLAIN SELECT l.cint, r.cint, l.cdecimal1, r.cdecimal2 + FROM decimal_mapjoin l + JOIN decimal_mapjoin r ON l.cint = r.cint + WHERE l.cint = 6981; +SELECT l.cint, r.cint, l.cdecimal1, r.cdecimal2 + FROM decimal_mapjoin l + JOIN decimal_mapjoin r ON l.cint = r.cint + WHERE l.cint = 6981; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q new file mode 100644 index 0000000000000..6e2c0b1fd403f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q @@ -0,0 +1,77 @@ +CREATE TABLE decimal_test STORED AS ORC AS SELECT cbigint, cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2 FROM alltypesorc; +SET hive.vectorized.execution.enabled=true; + +-- Test math functions in vectorized mode to verify they run correctly end-to-end. + +explain +select + cdecimal1 + ,Round(cdecimal1, 2) + ,Round(cdecimal1) + ,Floor(cdecimal1) + ,Ceil(cdecimal1) + ,Exp(cdecimal1) + ,Ln(cdecimal1) + ,Log10(cdecimal1) + -- Use log2 as a representative function to test all input types. + ,Log2(cdecimal1) + -- Use 15601.0 to test zero handling, as there are no zeroes in the table + ,Log2(cdecimal1 - 15601.0) + ,Log(2.0, cdecimal1) + ,Pow(log2(cdecimal1), 2.0) + ,Power(log2(cdecimal1), 2.0) + ,Sqrt(cdecimal1) + ,Abs(cdecimal1) + ,Sin(cdecimal1) + ,Asin(cdecimal1) + ,Cos(cdecimal1) + ,ACos(cdecimal1) + ,Atan(cdecimal1) + ,Degrees(cdecimal1) + ,Radians(cdecimal1) + ,Positive(cdecimal1) + ,Negative(cdecimal1) + ,Sign(cdecimal1) + -- Test nesting + ,cos(-sin(log(cdecimal1)) + 3.14159) +from decimal_test +-- limit output to a reasonably small number of rows +where cbigint % 500 = 0 +-- test use of a math function in the WHERE clause +and sin(cdecimal1) >= -1.0; + +select + cdecimal1 + ,Round(cdecimal1, 2) + ,Round(cdecimal1) + ,Floor(cdecimal1) + ,Ceil(cdecimal1) + ,Exp(cdecimal1) + ,Ln(cdecimal1) + ,Log10(cdecimal1) + -- Use log2 as a representative function to test all input types. + ,Log2(cdecimal1) + -- Use 15601.0 to test zero handling, as there are no zeroes in the table + ,Log2(cdecimal1 - 15601.0) + ,Log(2.0, cdecimal1) + ,Pow(log2(cdecimal1), 2.0) + ,Power(log2(cdecimal1), 2.0) + ,Sqrt(cdecimal1) + ,Abs(cdecimal1) + ,Sin(cdecimal1) + ,Asin(cdecimal1) + ,Cos(cdecimal1) + ,ACos(cdecimal1) + ,Atan(cdecimal1) + ,Degrees(cdecimal1) + ,Radians(cdecimal1) + ,Positive(cdecimal1) + ,Negative(cdecimal1) + ,Sign(cdecimal1) + -- Test nesting + ,cos(-sin(log(cdecimal1)) + 3.14159) +from decimal_test +-- limit output to a reasonably small number of rows +where cbigint % 500 = 0 +-- test use of a math function in the WHERE clause +and sin(cdecimal1) >= -1.0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q new file mode 100644 index 0000000000000..6e96690497390 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q @@ -0,0 +1,21 @@ +set hive.vectorized.execution.enabled=true; +set hive.auto.convert.join=true; +explain +select count(*) from (select c.ctinyint +from alltypesorc c +left outer join alltypesorc cd + on cd.cint = c.cint +left outer join alltypesorc hd + on hd.ctinyint = c.ctinyint +) t1 +; +select count(*) from (select c.ctinyint +from alltypesorc c +left outer join alltypesorc cd + on cd.cint = c.cint +left outer join alltypesorc hd + on hd.ctinyint = c.ctinyint +) t1; + +set hive.auto.convert.join=false; +set hive.vectorized.execution.enabled=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q new file mode 100644 index 0000000000000..fc1dc6d3b89f3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q @@ -0,0 +1,17 @@ +SET hive.vectorized.execution.enabled=true; +CREATE TABLE non_string_part(cint INT, cstring1 STRING, cdouble DOUBLE, ctimestamp1 TIMESTAMP) PARTITIONED BY (ctinyint tinyint) STORED AS ORC; +SET hive.exec.dynamic.partition.mode=nonstrict; +SET hive.exec.dynamic.partition=true; + +INSERT OVERWRITE TABLE non_string_part PARTITION(ctinyint) SELECT cint, cstring1, cdouble, ctimestamp1, ctinyint fROM alltypesorc +WHERE ctinyint IS NULL AND cdouble IS NOT NULL ORDER BY cdouble; + +SHOW PARTITIONS non_string_part; + +EXPLAIN SELECT cint, ctinyint FROM non_string_part WHERE cint > 0 ORDER BY cint LIMIT 10; + +SELECT cint, ctinyint FROM non_string_part WHERE cint > 0 ORDER BY cint LIMIT 10; + +EXPLAIN SELECT cint, cstring1 FROM non_string_part WHERE cint > 0 ORDER BY cint, cstring1 LIMIT 10; + +SELECT cint, cstring1 FROM non_string_part WHERE cint > 0 ORDER BY cint, cstring1 LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q new file mode 100644 index 0000000000000..39fba7d1ac32d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q @@ -0,0 +1,27 @@ +SET hive.vectorized.execution.enabled=true; +SELECT AVG(cbigint), + (-(AVG(cbigint))), + (-6432 + AVG(cbigint)), + STDDEV_POP(cbigint), + (-((-6432 + AVG(cbigint)))), + ((-((-6432 + AVG(cbigint)))) + (-6432 + AVG(cbigint))), + VAR_SAMP(cbigint), + (-((-6432 + AVG(cbigint)))), + (-6432 + (-((-6432 + AVG(cbigint))))), + (-((-6432 + AVG(cbigint)))), + ((-((-6432 + AVG(cbigint)))) / (-((-6432 + AVG(cbigint))))), + COUNT(*), + SUM(cfloat), + (VAR_SAMP(cbigint) % STDDEV_POP(cbigint)), + (-(VAR_SAMP(cbigint))), + ((-((-6432 + AVG(cbigint)))) * (-(AVG(cbigint)))), + MIN(ctinyint), + (-(MIN(ctinyint))) +FROM alltypesorc +WHERE (((cstring2 LIKE '%b%') + OR ((79.553 != cint) + OR (cbigint < cdouble))) + OR ((ctinyint >= csmallint) + AND ((cboolean2 = 1) + AND (3569 = ctinyint)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q new file mode 100644 index 0000000000000..745aa06f2f524 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q @@ -0,0 +1,21 @@ +SET hive.vectorized.execution.enabled=true; +SELECT VAR_POP(ctinyint), + (VAR_POP(ctinyint) / -26.28), + SUM(cfloat), + (-1.389 + SUM(cfloat)), + (SUM(cfloat) * (-1.389 + SUM(cfloat))), + MAX(ctinyint), + (-((SUM(cfloat) * (-1.389 + SUM(cfloat))))), + MAX(cint), + (MAX(cint) * 79.553), + VAR_SAMP(cdouble), + (10.175 % (-((SUM(cfloat) * (-1.389 + SUM(cfloat)))))), + COUNT(cint), + (-563 % MAX(cint)) +FROM alltypesorc +WHERE (((cdouble > ctinyint) + AND (cboolean2 > 0)) + OR ((cbigint < ctinyint) + OR ((cint > cbigint) + OR (cboolean1 < 0)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q new file mode 100644 index 0000000000000..720f38d02b072 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q @@ -0,0 +1,24 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cdouble, + ctimestamp1, + ctinyint, + cboolean1, + cstring1, + (-(cdouble)), + (cdouble + csmallint), + ((cdouble + csmallint) % 33), + (-(cdouble)), + (ctinyint % cdouble), + (ctinyint % csmallint), + (-(cdouble)), + (cbigint * (ctinyint % csmallint)), + (9763215.5639 - (cdouble + csmallint)), + (-((-(cdouble)))) +FROM alltypesorc +WHERE (((cstring2 <= '10') + OR ((ctinyint > cdouble) + AND (-5638.15 >= ctinyint))) + OR ((cdouble > 6981) + AND ((csmallint = 9763215.5639) + OR (cstring1 LIKE '%a')))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q new file mode 100644 index 0000000000000..329ed28948434 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q @@ -0,0 +1,15 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cstring1, + cboolean1, + cdouble, + ctimestamp1, + (-3728 * csmallint), + (cdouble - 9763215.5639), + (-(cdouble)), + ((-(cdouble)) + 6981), + (cdouble * -5638.15) +FROM alltypesorc +WHERE ((cstring2 = cstring1) + OR ((ctimestamp1 IS NULL) + AND (cstring1 LIKE '%a'))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q new file mode 100644 index 0000000000000..fef62fc7a0eb6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q @@ -0,0 +1,32 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cbigint, + cboolean1, + cstring1, + ctimestamp1, + cdouble, + (-6432 * cdouble), + (-(cbigint)), + COUNT(cbigint), + (cbigint * COUNT(cbigint)), + STDDEV_SAMP(cbigint), + ((-6432 * cdouble) / -6432), + (-(((-6432 * cdouble) / -6432))), + AVG(cdouble), + (-((-6432 * cdouble))), + (-5638.15 + cbigint), + SUM(cbigint), + (AVG(cdouble) / (-6432 * cdouble)), + AVG(cdouble), + (-((-(((-6432 * cdouble) / -6432))))), + (((-6432 * cdouble) / -6432) + (-((-6432 * cdouble)))), + STDDEV_POP(cdouble) +FROM alltypesorc +WHERE (((ctimestamp1 IS NULL) + AND ((cboolean1 >= cboolean2) + OR (ctinyint != csmallint))) + AND ((cstring1 LIKE '%a') + OR ((cboolean2 <= 1) + AND (cbigint >= csmallint)))) +GROUP BY cbigint, cboolean1, cstring1, ctimestamp1, cdouble +ORDER BY ctimestamp1, cdouble, cbigint, cstring1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q new file mode 100644 index 0000000000000..fad2585d22725 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q @@ -0,0 +1,31 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cboolean1, + ctinyint, + ctimestamp1, + cfloat, + cstring1, + (-(ctinyint)), + MAX(ctinyint), + ((-(ctinyint)) + MAX(ctinyint)), + SUM(cfloat), + (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))), + (-(SUM(cfloat))), + (79.553 * cfloat), + STDDEV_POP(cfloat), + (-(SUM(cfloat))), + STDDEV_POP(ctinyint), + (((-(ctinyint)) + MAX(ctinyint)) - 10.175), + (-((-(SUM(cfloat))))), + (-26.28 / (-((-(SUM(cfloat)))))), + MAX(cfloat), + ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint), + MIN(ctinyint) +FROM alltypesorc +WHERE (((cfloat < 3569) + AND ((10.175 >= cdouble) + AND (cboolean1 != 1))) + OR ((ctimestamp1 > -29071) + AND ((ctimestamp2 != -29071) + AND (ctinyint < 9763215.5639)))) +GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q new file mode 100644 index 0000000000000..a121c64edc7b4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q @@ -0,0 +1,33 @@ +SET hive.vectorized.execution.enabled=true; +SELECT ctimestamp1, + cfloat, + cstring1, + cboolean1, + cdouble, + (-26.28 + cdouble), + (-((-26.28 + cdouble))), + STDDEV_SAMP((-((-26.28 + cdouble)))), + (cfloat * -26.28), + MAX(cfloat), + (-(cfloat)), + (-(MAX(cfloat))), + ((-((-26.28 + cdouble))) / 10.175), + STDDEV_POP(cfloat), + COUNT(cfloat), + (-(((-((-26.28 + cdouble))) / 10.175))), + (-1.389 % STDDEV_SAMP((-((-26.28 + cdouble))))), + (cfloat - cdouble), + VAR_POP(cfloat), + (VAR_POP(cfloat) % 10.175), + VAR_SAMP(cfloat), + (-((cfloat - cdouble))) +FROM alltypesorc +WHERE (((ctinyint <= cbigint) + AND ((cint <= cdouble) + OR (ctimestamp2 < ctimestamp1))) + AND ((cdouble < ctinyint) + AND ((cbigint > -257) + OR (cfloat < cint)))) +GROUP BY ctimestamp1, cfloat, cstring1, cboolean1, cdouble +ORDER BY cstring1, cfloat, cdouble, ctimestamp1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q new file mode 100644 index 0000000000000..7daad0b2c50a5 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q @@ -0,0 +1,31 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cfloat, + cboolean1, + cdouble, + cstring1, + ctinyint, + cint, + ctimestamp1, + STDDEV_SAMP(cfloat), + (-26.28 - cint), + MIN(cdouble), + (cdouble * 79.553), + (33 % cfloat), + STDDEV_SAMP(ctinyint), + VAR_POP(ctinyint), + (-23 % cdouble), + (-(ctinyint)), + VAR_SAMP(cint), + (cint - cfloat), + (-23 % ctinyint), + (-((-26.28 - cint))), + STDDEV_POP(cint) +FROM alltypesorc +WHERE (((cstring2 LIKE '%ss%') + OR (cstring1 LIKE '10%')) + OR ((cint >= -75) + AND ((ctinyint = csmallint) + AND (cdouble >= -3728)))) +GROUP BY cfloat, cboolean1, cdouble, cstring1, ctinyint, cint, ctimestamp1 +ORDER BY cfloat, cboolean1, cdouble, cstring1, ctinyint, cint, ctimestamp1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q new file mode 100644 index 0000000000000..39a9402680496 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q @@ -0,0 +1,20 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cstring1, + cdouble, + ctimestamp1, + (cdouble - 9763215.5639), + (-((cdouble - 9763215.5639))), + COUNT(cdouble), + STDDEV_SAMP(cdouble), + (-(STDDEV_SAMP(cdouble))), + (STDDEV_SAMP(cdouble) * COUNT(cdouble)), + MIN(cdouble), + (9763215.5639 / cdouble), + (COUNT(cdouble) / -1.389), + STDDEV_SAMP(cdouble) +FROM alltypesorc +WHERE ((cstring2 LIKE '%b%') + AND ((cdouble >= -1.389) + OR (cstring1 < 'a'))) +GROUP BY cstring1, cdouble, ctimestamp1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q new file mode 100644 index 0000000000000..b8647a434414b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q @@ -0,0 +1,23 @@ +SET hive.vectorized.execution.enabled=true; +SELECT AVG(csmallint), + (AVG(csmallint) % -563), + (AVG(csmallint) + 762), + SUM(cfloat), + VAR_POP(cbigint), + (-(VAR_POP(cbigint))), + (SUM(cfloat) - AVG(csmallint)), + COUNT(*), + (-((SUM(cfloat) - AVG(csmallint)))), + (VAR_POP(cbigint) - 762), + MIN(ctinyint), + ((-(VAR_POP(cbigint))) + MIN(ctinyint)), + AVG(cdouble), + (((-(VAR_POP(cbigint))) + MIN(ctinyint)) - SUM(cfloat)) +FROM alltypesorc +WHERE (((ctimestamp1 < ctimestamp2) + AND ((cstring2 LIKE 'b%') + AND (cfloat <= -5638.15))) + OR ((cdouble < ctinyint) + AND ((-10669 != ctimestamp2) + OR (359 > cint)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q new file mode 100644 index 0000000000000..1d53994519db3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q @@ -0,0 +1,25 @@ +SET hive.vectorized.execution.enabled=true; +SELECT STDDEV_SAMP(csmallint), + (STDDEV_SAMP(csmallint) - 10.175), + STDDEV_POP(ctinyint), + (STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)), + (-(STDDEV_POP(ctinyint))), + (STDDEV_SAMP(csmallint) % 79.553), + (-((STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)))), + STDDEV_SAMP(cfloat), + (-(STDDEV_SAMP(csmallint))), + SUM(cfloat), + ((-((STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)))) / (STDDEV_SAMP(csmallint) - 10.175)), + (-((STDDEV_SAMP(csmallint) - 10.175))), + AVG(cint), + (-3728 - STDDEV_SAMP(csmallint)), + STDDEV_POP(cint), + (AVG(cint) / STDDEV_SAMP(cfloat)) +FROM alltypesorc +WHERE (((cint <= cfloat) + AND ((79.553 != cbigint) + AND (ctimestamp2 = -29071))) + OR ((cbigint > cdouble) + AND ((79.553 <= csmallint) + AND (ctimestamp1 > ctimestamp2)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q new file mode 100644 index 0000000000000..1eb324d1902e8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q @@ -0,0 +1,23 @@ +SET hive.vectorized.execution.enabled=true; +SELECT SUM(cint), + (SUM(cint) * -563), + (-3728 + SUM(cint)), + STDDEV_POP(cdouble), + (-(STDDEV_POP(cdouble))), + AVG(cdouble), + ((SUM(cint) * -563) % SUM(cint)), + (((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble)), + VAR_POP(cdouble), + (-((((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble)))), + ((-3728 + SUM(cint)) - (SUM(cint) * -563)), + MIN(ctinyint), + MIN(ctinyint), + (MIN(ctinyint) * (-((((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble))))) +FROM alltypesorc +WHERE (((csmallint >= cint) + OR ((-89010 >= ctinyint) + AND (cdouble > 79.553))) + OR ((-563 != cbigint) + AND ((ctinyint != cbigint) + OR (-3728 >= cdouble)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q new file mode 100644 index 0000000000000..826b20fd4f124 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q @@ -0,0 +1,20 @@ +SET hive.vectorized.execution.enabled=true; +SELECT MAX(csmallint), + (MAX(csmallint) * -75), + COUNT(*), + ((MAX(csmallint) * -75) / COUNT(*)), + (6981 * MAX(csmallint)), + MIN(csmallint), + (-(MIN(csmallint))), + (197 % ((MAX(csmallint) * -75) / COUNT(*))), + SUM(cint), + MAX(ctinyint), + (-(MAX(ctinyint))), + ((-(MAX(ctinyint))) + MAX(ctinyint)) +FROM alltypesorc +WHERE (((cboolean2 IS NOT NULL) + AND (cstring1 LIKE '%b%')) + OR ((ctinyint = cdouble) + AND ((ctimestamp2 IS NOT NULL) + AND (cstring2 LIKE 'a')))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q new file mode 100644 index 0000000000000..2b59f10ed89c3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q @@ -0,0 +1,21 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cboolean1, + cfloat, + cstring1, + (988888 * csmallint), + (-(csmallint)), + (-(cfloat)), + (-26.28 / cfloat), + (cfloat * 359), + (cint % ctinyint), + (-(cdouble)), + (ctinyint - -75), + (762 * (cint % ctinyint)) +FROM alltypesorc +WHERE ((ctinyint != 0) + AND ((((cboolean1 <= 0) + AND (cboolean2 >= cboolean1)) + OR ((cbigint IS NOT NULL) + AND ((cstring2 LIKE '%a') + OR (cfloat <= -257)))))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q new file mode 100644 index 0000000000000..20c1148659195 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q @@ -0,0 +1,25 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cboolean1, + cbigint, + csmallint, + ctinyint, + ctimestamp1, + cstring1, + (cbigint + cbigint), + (csmallint % -257), + (-(csmallint)), + (-(ctinyint)), + ((-(ctinyint)) + 17), + (cbigint * (-(csmallint))), + (cint % csmallint), + (-(ctinyint)), + ((-(ctinyint)) % ctinyint) +FROM alltypesorc +WHERE ((ctinyint != 0) + AND (((ctimestamp1 <= 0) + OR ((ctinyint = cint) + OR (cstring2 LIKE 'ss'))) + AND ((988888 < cdouble) + OR ((ctimestamp2 > -29071) + AND (3569 >= cdouble))))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q new file mode 100644 index 0000000000000..98b3385f165a9 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q @@ -0,0 +1,23 @@ +SET hive.vectorized.execution.enabled=true; +SELECT ctimestamp1, + cdouble, + cboolean1, + cstring1, + cfloat, + (-(cdouble)), + (-5638.15 - cdouble), + (cdouble * -257), + (cint + cfloat), + ((-(cdouble)) + cbigint), + (-(cdouble)), + (-1.389 - cfloat), + (-(cfloat)), + ((-5638.15 - cdouble) + (cint + cfloat)) +FROM alltypesorc +WHERE (((cstring2 IS NOT NULL) + AND ((ctimestamp1 <= -29071) + AND (ctimestamp2 != 16558))) + OR ((cfloat < -6432) + OR ((cboolean1 IS NOT NULL) + AND (cdouble = 988888)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q new file mode 100644 index 0000000000000..252c426d64a5e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q @@ -0,0 +1,24 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cfloat, + cstring1, + cint, + ctimestamp1, + cdouble, + cbigint, + (cfloat / ctinyint), + (cint % cbigint), + (-(cdouble)), + (cdouble + (cfloat / ctinyint)), + (cdouble / cint), + (-((-(cdouble)))), + (9763215.5639 % cbigint), + (2563.58 + (-((-(cdouble))))) +FROM alltypesorc +WHERE (((cbigint > -23) + AND ((cdouble != 988888) + OR (cint > -863.257))) + AND ((ctinyint >= 33) + OR ((csmallint >= cbigint) + OR (cfloat = cdouble)))) +ORDER BY cbigint, cfloat; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q new file mode 100644 index 0000000000000..2b82a5aa0452a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q @@ -0,0 +1,4 @@ +CREATE TABLE date_decimal_test STORED AS ORC AS SELECT cint, cdouble, CAST (CAST (cint AS TIMESTAMP) AS DATE) AS cdate, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal FROM alltypesorc; +SET hive.vectorized.execution.enabled=true; +EXPLAIN SELECT cdate, cdecimal from date_decimal_test where cint IS NOT NULL AND cdouble IS NOT NULL LIMIT 10; +SELECT cdate, cdecimal from date_decimal_test where cint IS NOT NULL AND cdouble IS NOT NULL LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q new file mode 100644 index 0000000000000..69e388a28e411 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q @@ -0,0 +1,24 @@ +SET hive.vectorized.execution.enabled = true; + +-- TODO: add more stuff here after HIVE-5918 is fixed, such as cbigint and constants +explain +select cdouble / 0.0 from alltypesorc limit 100; +select cdouble / 0.0 from alltypesorc limit 100; + +-- There are no zeros in the table, but there is 988888, so use it as zero + +-- TODO: add more stuff here after HIVE-5918 is fixed, such as cbigint and constants as numerators +explain +select (cbigint - 988888L) as s1, cdouble / (cbigint - 988888L) as s2, 1.2 / (cbigint - 988888L) +from alltypesorc where cbigint > 0 and cbigint < 100000000 order by s1, s2 limit 100; +select (cbigint - 988888L) as s1, cdouble / (cbigint - 988888L) as s2, 1.2 / (cbigint - 988888L) +from alltypesorc where cbigint > 0 and cbigint < 100000000 order by s1, s2 limit 100; + +-- There are no zeros in the table, but there is -200.0, so use it as zero + +explain +select (cdouble + 200.0) as s1, cbigint / (cdouble + 200.0) as s2, (cdouble + 200.0) / (cdouble + 200.0), cbigint / (cdouble + 200.0), 1 / (cdouble + 200.0), 1.2 / (cdouble + 200.0) +from alltypesorc where cdouble >= -500 and cdouble < -199 order by s1, s2 limit 100; +select (cdouble + 200.0) as s1, cbigint / (cdouble + 200.0) as s2, (cdouble + 200.0) / (cdouble + 200.0), cbigint / (cdouble + 200.0), 1 / (cdouble + 200.0), 1.2 / (cdouble + 200.0) +from alltypesorc where cdouble >= -500 and cdouble < -199 order by s1, s2 limit 100; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q new file mode 100644 index 0000000000000..094a8d26a3842 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q @@ -0,0 +1,37 @@ +SET hive.vectorized.execution.enabled=true; +explain SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7; +SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7; + +set hive.optimize.reducededuplication.min.reducer=1; +set hive.limit.pushdown.memory.usage=0.3f; + +-- HIVE-3562 Some limit can be pushed down to map stage - c/p parts from limit_pushdown + +explain +select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20; +select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20; + +-- deduped RS +explain +select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20; +select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20; + +-- distincts +explain +select distinct(ctinyint) from alltypesorc limit 20; +select distinct(ctinyint) from alltypesorc limit 20; + +explain +select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20; +select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20; + +-- limit zero +explain +select ctinyint,cdouble from alltypesorc order by ctinyint limit 0; +select ctinyint,cdouble from alltypesorc order by ctinyint limit 0; + +-- 2MR (applied to last RS) +explain +select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20; +select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q new file mode 100644 index 0000000000000..bb50f9b853287 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q @@ -0,0 +1,3 @@ +SET hive.vectorized.execution.enabled=true; +SELECT SUM(abs(ctinyint)) from alltypesorc; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q new file mode 100644 index 0000000000000..bfd3dd722190c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q @@ -0,0 +1,27 @@ +SET hive.vectorized.execution.enabled=true; +SELECT AVG(cbigint), + (-(AVG(cbigint))), + (-6432 + AVG(cbigint)), + STDDEV_POP(cbigint), + (-((-6432 + AVG(cbigint)))), + ((-((-6432 + AVG(cbigint)))) + (-6432 + AVG(cbigint))), + VAR_SAMP(cbigint), + (-((-6432 + AVG(cbigint)))), + (-6432 + (-((-6432 + AVG(cbigint))))), + (-((-6432 + AVG(cbigint)))), + ((-((-6432 + AVG(cbigint)))) / (-((-6432 + AVG(cbigint))))), + COUNT(*), + SUM(cfloat), + (VAR_SAMP(cbigint) % STDDEV_POP(cbigint)), + (-(VAR_SAMP(cbigint))), + ((-((-6432 + AVG(cbigint)))) * (-(AVG(cbigint)))), + MIN(ctinyint), + (-(MIN(ctinyint))) +FROM alltypesorc +WHERE (((cstring2 LIKE '%b%') + OR ((79.553 != cint) + OR (NOT(cbigint >= cdouble)))) + OR ((ctinyint >= csmallint) + AND (NOT ((cboolean2 != 1) + OR (3569 != ctinyint))))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q new file mode 100644 index 0000000000000..0e34585b5a8ba --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q @@ -0,0 +1,7 @@ +SET hive.vectorized.execution.enabled=true; +CREATE TABLE alltypesorc_part(ctinyint tinyint, csmallint smallint, cint int, cbigint bigint, cfloat float, cdouble double, cstring1 string, cstring2 string, ctimestamp1 timestamp, ctimestamp2 timestamp, cboolean1 boolean, cboolean2 boolean) partitioned by (ds string) STORED AS ORC; +insert overwrite table alltypesorc_part partition (ds='2011') select * from alltypesorc limit 100; +insert overwrite table alltypesorc_part partition (ds='2012') select * from alltypesorc limit 100; + +select count(cdouble), cint from alltypesorc_part where ds='2011' group by cint limit 10; +select count(*) from alltypesorc_part A join alltypesorc_part B on A.ds=B.ds; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q new file mode 100644 index 0000000000000..c68ce56fd47db --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q @@ -0,0 +1,7 @@ +SET hive.vectorized.execution.enabled=true; +CREATE TABLE alltypesorc_part(ctinyint tinyint, csmallint smallint, cint int, cbigint bigint, cfloat float, cdouble double, cstring1 string, cstring2 string, ctimestamp1 timestamp, ctimestamp2 timestamp, cboolean1 boolean, cboolean2 boolean) partitioned by (ds string) STORED AS ORC; +insert overwrite table alltypesorc_part partition (ds='2011') select * from alltypesorc limit 100; +insert overwrite table alltypesorc_part partition (ds='2012') select * from alltypesorc limit 100; + +explain select (cdouble+2) c1 from alltypesorc_part order by c1 limit 10; +select (cdouble+2) c1 from alltypesorc_part order by c1 limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q new file mode 100644 index 0000000000000..bafe5504789e8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q @@ -0,0 +1,4 @@ +SET hive.vectorized.execution.enabled=true; +SET hive.optimize.index.filter=true; +explain SELECT AVG(cbigint) FROM alltypesorc WHERE cbigint < cdouble; +SELECT AVG(cbigint) FROM alltypesorc WHERE cbigint < cdouble; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q new file mode 100644 index 0000000000000..638a31ff413b7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q @@ -0,0 +1,852 @@ +SET hive.vectorized.execution.enabled=true; + +-- If you look at ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/OrcFileGenerator.java +-- which is the data generation class you'll see that those values are specified in the +-- initializeFixedPointValues for each data type. When I created the queries I usedthose values +-- where I needed scalar values to ensure that when the queries executed their predicates would be +-- filtering on values that are guaranteed to exist. + +-- Beyond those values, all the other data in the alltypesorc file is random, but there is a +-- specific pattern to the data that is important for coverage. In orc and subsequently +-- vectorization there are a number of optimizations for certain data patterns: AllValues, NoNulls, +-- RepeatingValue, RepeatingNull. The data in alltypesorc is generated such that each column has +-- exactly 3 batches of each data pattern. This gives us coverage for the vector expression +-- optimizations and ensure the metadata in appropriately set on the row batch object which are +-- reused across batches. + +-- For the queries themselves in order to efficiently cover as much of the new vectorization +-- functionality as I could I used a number of different techniques to create the +-- vectorization_short_regress.q test suite, primarily equivalence classes, and pairwise +-- combinations. + +-- First I divided the search space into a number of dimensions such as type, aggregate function, +-- filter operation, arithmetic operation, etc. The types were explored as equivalence classes of +-- long, double, time, string, and bool. Also, rather than creating a very large number of small +-- queries the resulting vectors were grouped by compatible dimensions to reduce the number of +-- queries. + +-- TargetTypeClasses: Long, Timestamp, Double, String, Bool +-- Functions: Avg, Sum, StDevP, StDev, Var, Min, Count +-- ArithmeticOps: Add, Multiply, Subtract, Divide +-- FilterOps: Equal, NotEqual, GreaterThan, LessThan, LessThanOrEqual +-- GroupBy: NoGroupByProjectAggs +EXPLAIN SELECT AVG(cint), + (AVG(cint) + -3728), + (-((AVG(cint) + -3728))), + (-((-((AVG(cint) + -3728))))), + ((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)), + SUM(cdouble), + (-(AVG(cint))), + STDDEV_POP(cint), + (((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)) * (-((-((AVG(cint) + -3728)))))), + STDDEV_SAMP(csmallint), + (-(STDDEV_POP(cint))), + (STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))), + ((STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))) * STDDEV_POP(cint)), + VAR_SAMP(cint), + AVG(cfloat), + (10.175 - VAR_SAMP(cint)), + (-((10.175 - VAR_SAMP(cint)))), + ((-(STDDEV_POP(cint))) / -563), + STDDEV_SAMP(cint), + (-(((-(STDDEV_POP(cint))) / -563))), + (AVG(cint) / SUM(cdouble)), + MIN(ctinyint), + COUNT(csmallint), + (MIN(ctinyint) / ((-(STDDEV_POP(cint))) / -563)), + (-((AVG(cint) / SUM(cdouble)))) +FROM alltypesorc +WHERE ((762 = cbigint) + OR ((csmallint < cfloat) + AND ((ctimestamp2 > -10669) + AND (cdouble != cint))) + OR (cstring1 = 'a') + OR ((cbigint <= -1.389) + AND ((cstring2 != 'a') + AND ((79.553 != cint) + AND (cboolean2 != cboolean1))))); +SELECT AVG(cint), + (AVG(cint) + -3728), + (-((AVG(cint) + -3728))), + (-((-((AVG(cint) + -3728))))), + ((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)), + SUM(cdouble), + (-(AVG(cint))), + STDDEV_POP(cint), + (((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)) * (-((-((AVG(cint) + -3728)))))), + STDDEV_SAMP(csmallint), + (-(STDDEV_POP(cint))), + (STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))), + ((STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))) * STDDEV_POP(cint)), + VAR_SAMP(cint), + AVG(cfloat), + (10.175 - VAR_SAMP(cint)), + (-((10.175 - VAR_SAMP(cint)))), + ((-(STDDEV_POP(cint))) / -563), + STDDEV_SAMP(cint), + (-(((-(STDDEV_POP(cint))) / -563))), + (AVG(cint) / SUM(cdouble)), + MIN(ctinyint), + COUNT(csmallint), + (MIN(ctinyint) / ((-(STDDEV_POP(cint))) / -563)), + (-((AVG(cint) / SUM(cdouble)))) +FROM alltypesorc +WHERE ((762 = cbigint) + OR ((csmallint < cfloat) + AND ((ctimestamp2 > -10669) + AND (cdouble != cint))) + OR (cstring1 = 'a') + OR ((cbigint <= -1.389) + AND ((cstring2 != 'a') + AND ((79.553 != cint) + AND (cboolean2 != cboolean1))))); + +-- TargetTypeClasses: Long, Bool, Double, String, Timestamp +-- Functions: Max, VarP, StDevP, Avg, Min, StDev, Var +-- ArithmeticOps: Divide, Multiply, Remainder, Subtract +-- FilterOps: LessThan, LessThanOrEqual, GreaterThan, GreaterThanOrEqual, Like, RLike +-- GroupBy: NoGroupByProjectAggs +EXPLAIN SELECT MAX(cint), + (MAX(cint) / -3728), + (MAX(cint) * -3728), + VAR_POP(cbigint), + (-((MAX(cint) * -3728))), + STDDEV_POP(csmallint), + (-563 % (MAX(cint) * -3728)), + (VAR_POP(cbigint) / STDDEV_POP(csmallint)), + (-(STDDEV_POP(csmallint))), + MAX(cdouble), + AVG(ctinyint), + (STDDEV_POP(csmallint) - 10.175), + MIN(cint), + ((MAX(cint) * -3728) % (STDDEV_POP(csmallint) - 10.175)), + (-(MAX(cdouble))), + MIN(cdouble), + (MAX(cdouble) % -26.28), + STDDEV_SAMP(csmallint), + (-((MAX(cint) / -3728))), + ((-((MAX(cint) * -3728))) % (-563 % (MAX(cint) * -3728))), + ((MAX(cint) / -3728) - AVG(ctinyint)), + (-((MAX(cint) * -3728))), + VAR_SAMP(cint) +FROM alltypesorc +WHERE (((cbigint <= 197) + AND (cint < cbigint)) + OR ((cdouble >= -26.28) + AND (csmallint > cdouble)) + OR ((ctinyint > cfloat) + AND (cstring1 RLIKE '.*ss.*')) + OR ((cfloat > 79.553) + AND (cstring2 LIKE '10%'))); +SELECT MAX(cint), + (MAX(cint) / -3728), + (MAX(cint) * -3728), + VAR_POP(cbigint), + (-((MAX(cint) * -3728))), + STDDEV_POP(csmallint), + (-563 % (MAX(cint) * -3728)), + (VAR_POP(cbigint) / STDDEV_POP(csmallint)), + (-(STDDEV_POP(csmallint))), + MAX(cdouble), + AVG(ctinyint), + (STDDEV_POP(csmallint) - 10.175), + MIN(cint), + ((MAX(cint) * -3728) % (STDDEV_POP(csmallint) - 10.175)), + (-(MAX(cdouble))), + MIN(cdouble), + (MAX(cdouble) % -26.28), + STDDEV_SAMP(csmallint), + (-((MAX(cint) / -3728))), + ((-((MAX(cint) * -3728))) % (-563 % (MAX(cint) * -3728))), + ((MAX(cint) / -3728) - AVG(ctinyint)), + (-((MAX(cint) * -3728))), + VAR_SAMP(cint) +FROM alltypesorc +WHERE (((cbigint <= 197) + AND (cint < cbigint)) + OR ((cdouble >= -26.28) + AND (csmallint > cdouble)) + OR ((ctinyint > cfloat) + AND (cstring1 RLIKE '.*ss.*')) + OR ((cfloat > 79.553) + AND (cstring2 LIKE '10%'))); + +-- TargetTypeClasses: String, Long, Bool, Double, Timestamp +-- Functions: VarP, Count, Max, StDevP, StDev, Avg +-- ArithmeticOps: Subtract, Remainder, Multiply, Add +-- FilterOps: Equal, LessThanOrEqual, GreaterThan, Like, LessThan +-- GroupBy: NoGroupByProjectAggs +EXPLAIN SELECT VAR_POP(cbigint), + (-(VAR_POP(cbigint))), + (VAR_POP(cbigint) - (-(VAR_POP(cbigint)))), + COUNT(*), + (COUNT(*) % 79.553), + MAX(ctinyint), + (COUNT(*) - (-(VAR_POP(cbigint)))), + (-((-(VAR_POP(cbigint))))), + (-1 % (-(VAR_POP(cbigint)))), + COUNT(*), + (-(COUNT(*))), + STDDEV_POP(csmallint), + (-((-((-(VAR_POP(cbigint))))))), + (762 * (-(COUNT(*)))), + MAX(cint), + (MAX(ctinyint) + (762 * (-(COUNT(*))))), + ((-(VAR_POP(cbigint))) + MAX(cint)), + STDDEV_SAMP(cdouble), + ((-(COUNT(*))) % COUNT(*)), + COUNT(ctinyint), + AVG(ctinyint), + (-3728 % (MAX(ctinyint) + (762 * (-(COUNT(*)))))) +FROM alltypesorc +WHERE ((ctimestamp1 = ctimestamp2) + OR (762 = cfloat) + OR (cstring1 = 'ss') + OR ((csmallint <= cbigint) + AND (1 = cboolean2)) + OR ((cboolean1 IS NOT NULL) + AND ((ctimestamp2 IS NOT NULL) + AND (cstring2 > 'a')))); +SELECT VAR_POP(cbigint), + (-(VAR_POP(cbigint))), + (VAR_POP(cbigint) - (-(VAR_POP(cbigint)))), + COUNT(*), + (COUNT(*) % 79.553), + MAX(ctinyint), + (COUNT(*) - (-(VAR_POP(cbigint)))), + (-((-(VAR_POP(cbigint))))), + (-1 % (-(VAR_POP(cbigint)))), + COUNT(*), + (-(COUNT(*))), + STDDEV_POP(csmallint), + (-((-((-(VAR_POP(cbigint))))))), + (762 * (-(COUNT(*)))), + MAX(cint), + (MAX(ctinyint) + (762 * (-(COUNT(*))))), + ((-(VAR_POP(cbigint))) + MAX(cint)), + STDDEV_SAMP(cdouble), + ((-(COUNT(*))) % COUNT(*)), + COUNT(ctinyint), + AVG(ctinyint), + (-3728 % (MAX(ctinyint) + (762 * (-(COUNT(*)))))) +FROM alltypesorc +WHERE ((ctimestamp1 = ctimestamp2) + OR (762 = cfloat) + OR (cstring1 = 'ss') + OR ((csmallint <= cbigint) + AND (1 = cboolean2)) + OR ((cboolean1 IS NOT NULL) + AND ((ctimestamp2 IS NOT NULL) + AND (cstring2 > 'a')))); + +-- TargetTypeClasses: String, Bool, Timestamp, Long, Double +-- Functions: Avg, Max, StDev, VarP +-- ArithmeticOps: Add, Divide, Remainder, Multiply +-- FilterOps: LessThanOrEqual, NotEqual, GreaterThanOrEqual, LessThan, Equal +-- GroupBy: NoGroupByProjectAggs +EXPLAIN SELECT AVG(ctinyint), + (AVG(ctinyint) + 6981), + ((AVG(ctinyint) + 6981) + AVG(ctinyint)), + MAX(cbigint), + (((AVG(ctinyint) + 6981) + AVG(ctinyint)) / AVG(ctinyint)), + (-((AVG(ctinyint) + 6981))), + STDDEV_SAMP(cint), + (AVG(ctinyint) % (-((AVG(ctinyint) + 6981)))), + VAR_POP(cint), + VAR_POP(cbigint), + (-(MAX(cbigint))), + ((-(MAX(cbigint))) / STDDEV_SAMP(cint)), + MAX(cfloat), + (VAR_POP(cbigint) * -26.28) +FROM alltypesorc +WHERE (((ctimestamp2 <= ctimestamp1) + AND ((cbigint != cdouble) + AND ('ss' <= cstring1))) + OR ((csmallint < ctinyint) + AND (ctimestamp1 >= 0)) + OR (cfloat = 17)); +SELECT AVG(ctinyint), + (AVG(ctinyint) + 6981), + ((AVG(ctinyint) + 6981) + AVG(ctinyint)), + MAX(cbigint), + (((AVG(ctinyint) + 6981) + AVG(ctinyint)) / AVG(ctinyint)), + (-((AVG(ctinyint) + 6981))), + STDDEV_SAMP(cint), + (AVG(ctinyint) % (-((AVG(ctinyint) + 6981)))), + VAR_POP(cint), + VAR_POP(cbigint), + (-(MAX(cbigint))), + ((-(MAX(cbigint))) / STDDEV_SAMP(cint)), + MAX(cfloat), + (VAR_POP(cbigint) * -26.28) +FROM alltypesorc +WHERE (((ctimestamp2 <= ctimestamp1) + AND ((cbigint != cdouble) + AND ('ss' <= cstring1))) + OR ((csmallint < ctinyint) + AND (ctimestamp1 >= 0)) + OR (cfloat = 17)); + +-- TargetTypeClasses: Timestamp, String, Long, Double, Bool +-- Functions: Max, Avg, Min, Var, StDev, Count, StDevP, Sum +-- ArithmeticOps: Multiply, Subtract, Add, Divide +-- FilterOps: Like, NotEqual, LessThan, GreaterThanOrEqual, GreaterThan, RLike +-- GroupBy: NoGroupByProjectColumns +EXPLAIN SELECT cint, + cdouble, + ctimestamp2, + cstring1, + cboolean2, + ctinyint, + cfloat, + ctimestamp1, + csmallint, + cbigint, + (-3728 * cbigint), + (-(cint)), + (-863.257 - cint), + (-(csmallint)), + (csmallint - (-(csmallint))), + ((csmallint - (-(csmallint))) + (-(csmallint))), + (cint / cint), + ((-863.257 - cint) - -26.28), + (-(cfloat)), + (cdouble * -89010), + (ctinyint / 988888), + (-(ctinyint)), + (79.553 / ctinyint) +FROM alltypesorc +WHERE (((cstring1 RLIKE 'a.*') + AND (cstring2 LIKE '%ss%')) + OR ((1 != cboolean2) + AND ((csmallint < 79.553) + AND (-257 != ctinyint))) + OR ((cdouble > ctinyint) + AND (cfloat >= cint)) + OR ((cint < cbigint) + AND (ctinyint > cbigint))); +SELECT cint, + cdouble, + ctimestamp2, + cstring1, + cboolean2, + ctinyint, + cfloat, + ctimestamp1, + csmallint, + cbigint, + (-3728 * cbigint), + (-(cint)), + (-863.257 - cint), + (-(csmallint)), + (csmallint - (-(csmallint))), + ((csmallint - (-(csmallint))) + (-(csmallint))), + (cint / cint), + ((-863.257 - cint) - -26.28), + (-(cfloat)), + (cdouble * -89010), + (ctinyint / 988888), + (-(ctinyint)), + (79.553 / ctinyint) +FROM alltypesorc +WHERE (((cstring1 RLIKE 'a.*') + AND (cstring2 LIKE '%ss%')) + OR ((1 != cboolean2) + AND ((csmallint < 79.553) + AND (-257 != ctinyint))) + OR ((cdouble > ctinyint) + AND (cfloat >= cint)) + OR ((cint < cbigint) + AND (ctinyint > cbigint))); + +-- TargetTypeClasses: Long, String, Double, Bool, Timestamp +-- Functions: VarP, Var, StDev, StDevP, Max, Sum +-- ArithmeticOps: Divide, Remainder, Subtract, Multiply +-- FilterOps: Equal, LessThanOrEqual, LessThan, Like, GreaterThanOrEqual, NotEqual, GreaterThan +-- GroupBy: NoGroupByProjectColumns +EXPLAIN SELECT cint, + cbigint, + cstring1, + cboolean1, + cfloat, + cdouble, + ctimestamp2, + csmallint, + cstring2, + cboolean2, + (cint / cbigint), + (cbigint % 79.553), + (-((cint / cbigint))), + (10.175 % cfloat), + (-(cfloat)), + (cfloat - (-(cfloat))), + ((cfloat - (-(cfloat))) % -6432), + (cdouble * csmallint), + (-(cdouble)), + (-(cbigint)), + (cfloat - (cint / cbigint)), + (-(csmallint)), + (3569 % cbigint), + (359 - cdouble), + (-(csmallint)) +FROM alltypesorc +WHERE (((197 > ctinyint) + AND (cint = cbigint)) + OR (cbigint = 359) + OR (cboolean1 < 0) + OR ((cstring1 LIKE '%ss') + AND (cfloat <= ctinyint))); + +SELECT cint, + cbigint, + cstring1, + cboolean1, + cfloat, + cdouble, + ctimestamp2, + csmallint, + cstring2, + cboolean2, + (cint / cbigint), + (cbigint % 79.553), + (-((cint / cbigint))), + (10.175 % cfloat), + (-(cfloat)), + (cfloat - (-(cfloat))), + ((cfloat - (-(cfloat))) % -6432), + (cdouble * csmallint), + (-(cdouble)), + (-(cbigint)), + (cfloat - (cint / cbigint)), + (-(csmallint)), + (3569 % cbigint), + (359 - cdouble), + (-(csmallint)) +FROM alltypesorc +WHERE (((197 > ctinyint) + AND (cint = cbigint)) + OR (cbigint = 359) + OR (cboolean1 < 0) + OR ((cstring1 LIKE '%ss') + AND (cfloat <= ctinyint))); + +-- TargetTypeClasses: String, Bool, Double, Long, Timestamp +-- Functions: Sum, Max, Avg, Var, StDevP, VarP +-- ArithmeticOps: Add, Subtract, Divide, Multiply, Remainder +-- FilterOps: NotEqual, GreaterThanOrEqual, Like, LessThanOrEqual, Equal, GreaterThan +-- GroupBy: NoGroupByProjectColumns +EXPLAIN SELECT cint, + cstring1, + cboolean2, + ctimestamp2, + cdouble, + cfloat, + cbigint, + csmallint, + cboolean1, + (cint + csmallint), + (cbigint - ctinyint), + (-(cbigint)), + (-(cfloat)), + ((cbigint - ctinyint) + cbigint), + (cdouble / cdouble), + (-(cdouble)), + ((cint + csmallint) * (-(cbigint))), + ((-(cdouble)) + cbigint), + (-1.389 / ctinyint), + (cbigint % cdouble), + (-(csmallint)), + (csmallint + (cint + csmallint)) +FROM alltypesorc +WHERE (((csmallint > -26.28) + AND (cstring2 LIKE 'ss')) + OR ((cdouble <= cbigint) + AND ((cstring1 >= 'ss') + AND (cint != cdouble))) + OR (ctinyint = -89010) + OR ((cbigint <= cfloat) + AND (-26.28 <= csmallint))) +ORDER BY cboolean1, cstring1, ctimestamp2, cfloat, cbigint, cstring1, cdouble, cint, csmallint, cdouble; +SELECT cint, + cstring1, + cboolean2, + ctimestamp2, + cdouble, + cfloat, + cbigint, + csmallint, + cboolean1, + (cint + csmallint), + (cbigint - ctinyint), + (-(cbigint)), + (-(cfloat)), + ((cbigint - ctinyint) + cbigint), + (cdouble / cdouble), + (-(cdouble)), + ((cint + csmallint) * (-(cbigint))), + ((-(cdouble)) + cbigint), + (-1.389 / ctinyint), + (cbigint % cdouble), + (-(csmallint)), + (csmallint + (cint + csmallint)) +FROM alltypesorc +WHERE (((csmallint > -26.28) + AND (cstring2 LIKE 'ss')) + OR ((cdouble <= cbigint) + AND ((cstring1 >= 'ss') + AND (cint != cdouble))) + OR (ctinyint = -89010) + OR ((cbigint <= cfloat) + AND (-26.28 <= csmallint))) +ORDER BY cboolean1, cstring1, ctimestamp2, cfloat, cbigint, cstring1, cdouble, cint, csmallint, cdouble; + +-- TargetTypeClasses: Long, String, Double, Timestamp +-- Functions: Avg, Min, StDevP, Sum, Var +-- ArithmeticOps: Divide, Subtract, Multiply, Remainder +-- FilterOps: GreaterThan, LessThan, LessThanOrEqual, GreaterThanOrEqual, Like +-- GroupBy: NoGroupByProjectColumns +EXPLAIN SELECT ctimestamp1, + cstring2, + cdouble, + cfloat, + cbigint, + csmallint, + (cbigint / 3569), + (-257 - csmallint), + (-6432 * cfloat), + (-(cdouble)), + (cdouble * 10.175), + ((-6432 * cfloat) / cfloat), + (-(cfloat)), + (cint % csmallint), + (-(cdouble)), + (cdouble * (-(cdouble))) +FROM alltypesorc +WHERE (((-1.389 >= cint) + AND ((csmallint < ctinyint) + AND (-6432 > csmallint))) + OR ((cdouble >= cfloat) + AND (cstring2 <= 'a')) + OR ((cstring1 LIKE 'ss%') + AND (10.175 > cbigint))) +ORDER BY csmallint, cstring2, cdouble; +SELECT ctimestamp1, + cstring2, + cdouble, + cfloat, + cbigint, + csmallint, + (cbigint / 3569), + (-257 - csmallint), + (-6432 * cfloat), + (-(cdouble)), + (cdouble * 10.175), + ((-6432 * cfloat) / cfloat), + (-(cfloat)), + (cint % csmallint), + (-(cdouble)), + (cdouble * (-(cdouble))) +FROM alltypesorc +WHERE (((-1.389 >= cint) + AND ((csmallint < ctinyint) + AND (-6432 > csmallint))) + OR ((cdouble >= cfloat) + AND (cstring2 <= 'a')) + OR ((cstring1 LIKE 'ss%') + AND (10.175 > cbigint))) +ORDER BY csmallint, cstring2, cdouble; + +-- TargetTypeClasses: Double, String, Long +-- Functions: StDev, Sum, VarP, Count +-- ArithmeticOps: Remainder, Divide, Subtract +-- FilterOps: GreaterThanOrEqual, Equal, LessThanOrEqual +-- GroupBy: GroupBy +EXPLAIN SELECT csmallint, + (csmallint % -75), + STDDEV_SAMP(csmallint), + (-1.389 / csmallint), + SUM(cbigint), + ((csmallint % -75) / SUM(cbigint)), + (-((csmallint % -75))), + VAR_POP(ctinyint), + (-((-((csmallint % -75))))), + COUNT(*), + (COUNT(*) - -89010) +FROM alltypesorc +WHERE (((csmallint >= -257)) + AND ((-6432 = csmallint) + OR ((cint >= cdouble) + AND (ctinyint <= cint)))) +GROUP BY csmallint +ORDER BY csmallint; +SELECT csmallint, + (csmallint % -75), + STDDEV_SAMP(csmallint), + (-1.389 / csmallint), + SUM(cbigint), + ((csmallint % -75) / SUM(cbigint)), + (-((csmallint % -75))), + VAR_POP(ctinyint), + (-((-((csmallint % -75))))), + COUNT(*), + (COUNT(*) - -89010) +FROM alltypesorc +WHERE (((csmallint >= -257)) + AND ((-6432 = csmallint) + OR ((cint >= cdouble) + AND (ctinyint <= cint)))) +GROUP BY csmallint +ORDER BY csmallint; + +-- TargetTypeClasses: Long, Double, Timestamp +-- Functions: Var, Count, Sum, VarP, StDevP +-- ArithmeticOps: Multiply, Add, Subtract, Remainder +-- FilterOps: GreaterThan, LessThan, Equal, LessThanOrEqual, GreaterThanOrEqual +-- GroupBy: GroupBy +EXPLAIN SELECT cdouble, + VAR_SAMP(cdouble), + (2563.58 * VAR_SAMP(cdouble)), + (-(VAR_SAMP(cdouble))), + COUNT(cfloat), + ((2563.58 * VAR_SAMP(cdouble)) + -5638.15), + ((-(VAR_SAMP(cdouble))) * ((2563.58 * VAR_SAMP(cdouble)) + -5638.15)), + SUM(cfloat), + VAR_POP(cdouble), + (cdouble - (-(VAR_SAMP(cdouble)))), + STDDEV_POP(cdouble), + (cdouble + VAR_SAMP(cdouble)), + (cdouble * 762), + SUM(cdouble), + (-863.257 % (cdouble * 762)), + SUM(cdouble) +FROM alltypesorc +WHERE (((cdouble > 2563.58)) + AND (((cbigint >= cint) + AND ((csmallint < cint) + AND (cfloat < -5638.15))) + OR (2563.58 = ctinyint) + OR ((cdouble <= cbigint) + AND (-5638.15 > cbigint)))) +GROUP BY cdouble +ORDER BY cdouble; +SELECT cdouble, + VAR_SAMP(cdouble), + (2563.58 * VAR_SAMP(cdouble)), + (-(VAR_SAMP(cdouble))), + COUNT(cfloat), + ((2563.58 * VAR_SAMP(cdouble)) + -5638.15), + ((-(VAR_SAMP(cdouble))) * ((2563.58 * VAR_SAMP(cdouble)) + -5638.15)), + SUM(cfloat), + VAR_POP(cdouble), + (cdouble - (-(VAR_SAMP(cdouble)))), + STDDEV_POP(cdouble), + (cdouble + VAR_SAMP(cdouble)), + (cdouble * 762), + SUM(cdouble), + (-863.257 % (cdouble * 762)), + SUM(cdouble) +FROM alltypesorc +WHERE (((cdouble > 2563.58)) + AND (((cbigint >= cint) + AND ((csmallint < cint) + AND (cfloat < -5638.15))) + OR (2563.58 = ctinyint) + OR ((cdouble <= cbigint) + AND (-5638.15 > cbigint)))) +GROUP BY cdouble +ORDER BY cdouble; + +-- TargetTypeClasses: Bool, Timestamp, String, Double, Long +-- Functions: StDevP, Avg, Count, Min, Var, VarP, Sum +-- ArithmeticOps: Multiply, Subtract, Add, Divide, Remainder +-- FilterOps: NotEqual, LessThan, Like, Equal, RLike +-- GroupBy: GroupBy +EXPLAIN SELECT ctimestamp1, + cstring1, + STDDEV_POP(cint), + (STDDEV_POP(cint) * 10.175), + (-(STDDEV_POP(cint))), + AVG(csmallint), + (-(STDDEV_POP(cint))), + (-26.28 - STDDEV_POP(cint)), + COUNT(*), + (-(COUNT(*))), + ((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))), + MIN(ctinyint), + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*)))), + (-((STDDEV_POP(cint) * 10.175))), + VAR_SAMP(csmallint), + (VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))), + (-((-(STDDEV_POP(cint))))), + ((-(COUNT(*))) / STDDEV_POP(cint)), + VAR_POP(cfloat), + (10.175 / AVG(csmallint)), + AVG(cint), + VAR_SAMP(cfloat), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))), + (-((-((STDDEV_POP(cint) * 10.175))))), + AVG(cfloat), + (((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) * 10.175), + (10.175 % (10.175 / AVG(csmallint))), + (-(MIN(ctinyint))), + MIN(cdouble), + VAR_POP(csmallint), + (-(((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))))), + ((-(STDDEV_POP(cint))) % AVG(cfloat)), + (-26.28 / (-(MIN(ctinyint)))), + STDDEV_POP(ctinyint), + SUM(cint), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) / VAR_POP(cfloat)), + (-((-(COUNT(*))))), + COUNT(*), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) % -26.28) +FROM alltypesorc +WHERE (((ctimestamp1 != 0)) + AND ((((-257 != ctinyint) + AND (cboolean2 IS NOT NULL)) + AND ((cstring1 RLIKE '.*ss') + AND (-10669 < ctimestamp1))) + OR (ctimestamp2 = -10669) + OR ((ctimestamp1 < 0) + AND (cstring2 LIKE '%b%')) + OR (cdouble = cint) + OR ((cboolean1 IS NULL) + AND (cfloat < cint)))) +GROUP BY ctimestamp1, cstring1; +SELECT ctimestamp1, + cstring1, + STDDEV_POP(cint), + (STDDEV_POP(cint) * 10.175), + (-(STDDEV_POP(cint))), + AVG(csmallint), + (-(STDDEV_POP(cint))), + (-26.28 - STDDEV_POP(cint)), + COUNT(*), + (-(COUNT(*))), + ((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))), + MIN(ctinyint), + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*)))), + (-((STDDEV_POP(cint) * 10.175))), + VAR_SAMP(csmallint), + (VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))), + (-((-(STDDEV_POP(cint))))), + ((-(COUNT(*))) / STDDEV_POP(cint)), + VAR_POP(cfloat), + (10.175 / AVG(csmallint)), + AVG(cint), + VAR_SAMP(cfloat), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))), + (-((-((STDDEV_POP(cint) * 10.175))))), + AVG(cfloat), + (((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) * 10.175), + (10.175 % (10.175 / AVG(csmallint))), + (-(MIN(ctinyint))), + MIN(cdouble), + VAR_POP(csmallint), + (-(((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))))), + ((-(STDDEV_POP(cint))) % AVG(cfloat)), + (-26.28 / (-(MIN(ctinyint)))), + STDDEV_POP(ctinyint), + SUM(cint), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) / VAR_POP(cfloat)), + (-((-(COUNT(*))))), + COUNT(*), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) % -26.28) +FROM alltypesorc +WHERE (((ctimestamp1 != 0)) + AND ((((-257 != ctinyint) + AND (cboolean2 IS NOT NULL)) + AND ((cstring1 RLIKE '.*ss') + AND (-10669 < ctimestamp1))) + OR (ctimestamp2 = -10669) + OR ((ctimestamp1 < 0) + AND (cstring2 LIKE '%b%')) + OR (cdouble = cint) + OR ((cboolean1 IS NULL) + AND (cfloat < cint)))) +GROUP BY ctimestamp1, cstring1; + +-- TargetTypeClasses: Double, Long, String, Timestamp, Bool +-- Functions: Max, Sum, Var, Avg, Min, VarP, StDev, StDevP +-- ArithmeticOps: Divide, Subtract, Remainder, Add, Multiply +-- FilterOps: GreaterThan, LessThanOrEqual, Equal, LessThan, GreaterThanOrEqual, NotEqual, Like, RLike +-- GroupBy: GroupBy +EXPLAIN SELECT cboolean1, + MAX(cfloat), + (-(MAX(cfloat))), + (-26.28 / MAX(cfloat)), + SUM(cbigint), + (SUM(cbigint) - 10.175), + VAR_SAMP(cint), + (VAR_SAMP(cint) % MAX(cfloat)), + (10.175 + (-(MAX(cfloat)))), + AVG(cdouble), + ((SUM(cbigint) - 10.175) + VAR_SAMP(cint)), + MIN(cbigint), + VAR_POP(cbigint), + (-((10.175 + (-(MAX(cfloat)))))), + (79.553 / VAR_POP(cbigint)), + (VAR_SAMP(cint) % (79.553 / VAR_POP(cbigint))), + (-((10.175 + (-(MAX(cfloat)))))), + SUM(cint), + STDDEV_SAMP(ctinyint), + (-1.389 * MIN(cbigint)), + (SUM(cint) - (-1.389 * MIN(cbigint))), + STDDEV_POP(csmallint), + (-((SUM(cint) - (-1.389 * MIN(cbigint))))), + AVG(cint), + (-(AVG(cint))), + (AVG(cint) * SUM(cint)) +FROM alltypesorc +WHERE (((cboolean1 IS NOT NULL)) + AND (((cdouble < csmallint) + AND ((cboolean2 = cboolean1) + AND (cbigint <= -863.257))) + OR ((cint >= -257) + AND ((cstring1 IS NOT NULL) + AND (cboolean1 >= 1))) + OR (cstring2 RLIKE 'b') + OR ((csmallint >= ctinyint) + AND (ctimestamp2 IS NULL)))) +GROUP BY cboolean1 +ORDER BY cboolean1; +SELECT cboolean1, + MAX(cfloat), + (-(MAX(cfloat))), + (-26.28 / MAX(cfloat)), + SUM(cbigint), + (SUM(cbigint) - 10.175), + VAR_SAMP(cint), + (VAR_SAMP(cint) % MAX(cfloat)), + (10.175 + (-(MAX(cfloat)))), + AVG(cdouble), + ((SUM(cbigint) - 10.175) + VAR_SAMP(cint)), + MIN(cbigint), + VAR_POP(cbigint), + (-((10.175 + (-(MAX(cfloat)))))), + (79.553 / VAR_POP(cbigint)), + (VAR_SAMP(cint) % (79.553 / VAR_POP(cbigint))), + (-((10.175 + (-(MAX(cfloat)))))), + SUM(cint), + STDDEV_SAMP(ctinyint), + (-1.389 * MIN(cbigint)), + (SUM(cint) - (-1.389 * MIN(cbigint))), + STDDEV_POP(csmallint), + (-((SUM(cint) - (-1.389 * MIN(cbigint))))), + AVG(cint), + (-(AVG(cint))), + (AVG(cint) * SUM(cint)) +FROM alltypesorc +WHERE (((cboolean1 IS NOT NULL)) + AND (((cdouble < csmallint) + AND ((cboolean2 = cboolean1) + AND (cbigint <= -863.257))) + OR ((cint >= -257) + AND ((cstring1 IS NOT NULL) + AND (cboolean1 >= 1))) + OR (cstring2 RLIKE 'b') + OR ((csmallint >= ctinyint) + AND (ctimestamp2 IS NULL)))) +GROUP BY cboolean1 +ORDER BY cboolean1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q new file mode 100644 index 0000000000000..e309713795af8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q @@ -0,0 +1,46 @@ +create table vsmb_bucket_1(key int, value string) + CLUSTERED BY (key) + SORTED BY (key) INTO 1 BUCKETS + STORED AS ORC; +create table vsmb_bucket_2(key int, value string) + CLUSTERED BY (key) + SORTED BY (key) INTO 1 BUCKETS + STORED AS ORC; + +create table vsmb_bucket_RC(key int, value string) + CLUSTERED BY (key) + SORTED BY (key) INTO 1 BUCKETS + STORED AS RCFILE; + +create table vsmb_bucket_TXT(key int, value string) + CLUSTERED BY (key) + SORTED BY (key) INTO 1 BUCKETS + STORED AS TEXTFILE; + +insert into table vsmb_bucket_1 select cint, cstring1 from alltypesorc limit 2; +insert into table vsmb_bucket_2 select cint, cstring1 from alltypesorc limit 2; +insert into table vsmb_bucket_RC select cint, cstring1 from alltypesorc limit 2; +insert into table vsmb_bucket_TXT select cint, cstring1 from alltypesorc limit 2; + +set hive.vectorized.execution.enabled=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.auto.convert.sortmerge.join.noconditionaltask = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +select /*+MAPJOIN(a)*/ * from vsmb_bucket_1 a join vsmb_bucket_2 b on a.key = b.key; +select /*+MAPJOIN(a)*/ * from vsmb_bucket_1 a join vsmb_bucket_2 b on a.key = b.key; + +explain +select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_RC b on a.key = b.key; +select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_RC b on a.key = b.key; + +-- RC file does not yet provide the vectorized CommonRCFileformat out-of-the-box +-- explain +-- select /*+MAPJOIN(b)*/ * from vsmb_bucket_RC a join vsmb_bucket_2 b on a.key = b.key; +-- select /*+MAPJOIN(b)*/ * from vsmb_bucket_RC a join vsmb_bucket_2 b on a.key = b.key; + +explain +select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_TXT b on a.key = b.key; +select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_TXT b on a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q new file mode 100644 index 0000000000000..e448d51f6bc30 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q @@ -0,0 +1,37 @@ +set hive.vectorized.execution.enabled = true +; +explain +select + csmallint, + case + when csmallint = 418 then "a" + when csmallint = 12205 then "b" + else "c" + end, + case csmallint + when 418 then "a" + when 12205 then "b" + else "c" + end +from alltypesorc +where csmallint = 418 +or csmallint = 12205 +or csmallint = 10583 +; +select + csmallint, + case + when csmallint = 418 then "a" + when csmallint = 12205 then "b" + else "c" + end, + case csmallint + when 418 then "a" + when 12205 then "b" + else "c" + end +from alltypesorc +where csmallint = 418 +or csmallint = 12205 +or csmallint = 10583 +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q new file mode 100644 index 0000000000000..3f818b18534af --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q @@ -0,0 +1,149 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test type casting in vectorized mode to verify end-to-end functionality. + +explain +select +-- to boolean + cast (ctinyint as boolean) + ,cast (csmallint as boolean) + ,cast (cint as boolean) + ,cast (cbigint as boolean) + ,cast (cfloat as boolean) + ,cast (cdouble as boolean) + ,cast (cboolean1 as boolean) + ,cast (cbigint * 0 as boolean) + ,cast (ctimestamp1 as boolean) + ,cast (cstring1 as boolean) +-- to int family + ,cast (ctinyint as int) + ,cast (csmallint as int) + ,cast (cint as int) + ,cast (cbigint as int) + ,cast (cfloat as int) + ,cast (cdouble as int) + ,cast (cboolean1 as int) + ,cast (ctimestamp1 as int) + ,cast (cstring1 as int) + ,cast (substr(cstring1, 1, 1) as int) + ,cast (cfloat as tinyint) + ,cast (cfloat as smallint) + ,cast (cfloat as bigint) +-- to float family + ,cast (ctinyint as double) + ,cast (csmallint as double) + ,cast (cint as double) + ,cast (cbigint as double) + ,cast (cfloat as double) + ,cast (cdouble as double) + ,cast (cboolean1 as double) + ,cast (ctimestamp1 as double) + ,cast (cstring1 as double) + ,cast (substr(cstring1, 1, 1) as double) + ,cast (cint as float) + ,cast (cdouble as float) +-- to timestamp + ,cast (ctinyint as timestamp) + ,cast (csmallint as timestamp) + ,cast (cint as timestamp) + ,cast (cbigint as timestamp) + ,cast (cfloat as timestamp) + ,cast (cdouble as timestamp) + ,cast (cboolean1 as timestamp) + ,cast (cbigint * 0 as timestamp) + ,cast (ctimestamp1 as timestamp) + ,cast (cstring1 as timestamp) + ,cast (substr(cstring1, 1, 1) as timestamp) +-- to string + ,cast (ctinyint as string) + ,cast (csmallint as string) + ,cast (cint as string) + ,cast (cbigint as string) + ,cast (cfloat as string) + ,cast (cdouble as string) + ,cast (cboolean1 as string) + ,cast (cbigint * 0 as string) + ,cast (ctimestamp1 as string) + ,cast (cstring1 as string) +-- nested and expression arguments + ,cast (cast (cfloat as int) as float) + ,cast (cint * 2 as double) + ,cast (sin(cfloat) as string) + ,cast (cint as float) + cast(cboolean1 as double) +from alltypesorc +-- limit output to a reasonably small number of rows +where cbigint % 250 = 0; + + +select +-- to boolean + cast (ctinyint as boolean) + ,cast (csmallint as boolean) + ,cast (cint as boolean) + ,cast (cbigint as boolean) + ,cast (cfloat as boolean) + ,cast (cdouble as boolean) + ,cast (cboolean1 as boolean) + ,cast (cbigint * 0 as boolean) + ,cast (ctimestamp1 as boolean) + ,cast (cstring1 as boolean) +-- to int family + ,cast (ctinyint as int) + ,cast (csmallint as int) + ,cast (cint as int) + ,cast (cbigint as int) + ,cast (cfloat as int) + ,cast (cdouble as int) + ,cast (cboolean1 as int) + ,cast (ctimestamp1 as int) + ,cast (cstring1 as int) + ,cast (substr(cstring1, 1, 1) as int) + ,cast (cfloat as tinyint) + ,cast (cfloat as smallint) + ,cast (cfloat as bigint) +-- to float family + ,cast (ctinyint as double) + ,cast (csmallint as double) + ,cast (cint as double) + ,cast (cbigint as double) + ,cast (cfloat as double) + ,cast (cdouble as double) + ,cast (cboolean1 as double) + ,cast (ctimestamp1 as double) + ,cast (cstring1 as double) + ,cast (substr(cstring1, 1, 1) as double) + ,cast (cint as float) + ,cast (cdouble as float) +-- to timestamp + ,cast (ctinyint as timestamp) + ,cast (csmallint as timestamp) + ,cast (cint as timestamp) + ,cast (cbigint as timestamp) + ,cast (cfloat as timestamp) + ,cast (cdouble as timestamp) + ,cast (cboolean1 as timestamp) + ,cast (cbigint * 0 as timestamp) + ,cast (ctimestamp1 as timestamp) + ,cast (cstring1 as timestamp) + ,cast (substr(cstring1, 1, 1) as timestamp) +-- to string + ,cast (ctinyint as string) + ,cast (csmallint as string) + ,cast (cint as string) + ,cast (cbigint as string) + ,cast (cfloat as string) + ,cast (cdouble as string) + ,cast (cboolean1 as string) + ,cast (cbigint * 0 as string) + ,cast (ctimestamp1 as string) + ,cast (cstring1 as string) +-- nested and expression arguments + ,cast (cast (cfloat as int) as float) + ,cast (cint * 2 as double) + ,cast (sin(cfloat) as string) + ,cast (cint as float) + cast(cboolean1 as double) +from alltypesorc +-- limit output to a reasonably small number of rows +where cbigint % 250 = 0; + + \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q new file mode 100644 index 0000000000000..381e4255ca51c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q @@ -0,0 +1,47 @@ +create table store(s_store_sk int, s_city string) +stored as orc; +insert overwrite table store +select cint, cstring1 +from alltypesorc +where cint not in ( +-3728, -563, 762, 6981, 253665376, 528534767, 626923679); +create table store_sales(ss_store_sk int, ss_hdemo_sk int, ss_net_profit double) +stored as orc; +insert overwrite table store_sales +select cint, cint, cdouble +from alltypesorc +where cint not in ( +-3728, -563, 762, 6981, 253665376, 528534767, 626923679); +create table household_demographics(hd_demo_sk int) +stored as orc; +insert overwrite table household_demographics +select cint +from alltypesorc +where cint not in ( +-3728, -563, 762, 6981, 253665376, 528534767, 626923679); +set hive.auto.convert.join=true; +set hive.vectorized.execution.enabled=true; + + +explain +select store.s_city, ss_net_profit +from store_sales +JOIN store ON store_sales.ss_store_sk = store.s_store_sk +JOIN household_demographics ON store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk +limit 100 +; + +select store.s_city, ss_net_profit +from store_sales +JOIN store ON store_sales.ss_store_sk = store.s_store_sk +JOIN household_demographics ON store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk +limit 100 +; + +set hive.auto.convert.join=false; +set hive.vectorized.execution.enabled=false; + +drop table store; +drop table store_sales; +drop table household_demographics; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q new file mode 100644 index 0000000000000..b7aa3c28ac059 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q @@ -0,0 +1,122 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test timestamp functions in vectorized mode to verify they run correctly end-to-end. + +CREATE TABLE date_udf_flight ( + origin_city_name STRING, + dest_city_name STRING, + fl_date DATE, + arr_delay FLOAT, + fl_num INT +); +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight; + +CREATE TABLE date_udf_flight_orc ( + fl_date DATE, + fl_time TIMESTAMP +) STORED AS ORC; + +INSERT INTO TABLE date_udf_flight_orc SELECT fl_date, to_utc_timestamp(fl_date, 'America/Los_Angeles') FROM date_udf_flight; + +SELECT * FROM date_udf_flight_orc; + +EXPLAIN SELECT + to_unix_timestamp(fl_time), + year(fl_time), + month(fl_time), + day(fl_time), + dayofmonth(fl_time), + weekofyear(fl_time), + date(fl_time), + to_date(fl_time), + date_add(fl_time, 2), + date_sub(fl_time, 2), + datediff(fl_time, "2000-01-01") +FROM date_udf_flight_orc; + +SELECT + to_unix_timestamp(fl_time), + year(fl_time), + month(fl_time), + day(fl_time), + dayofmonth(fl_time), + weekofyear(fl_time), + date(fl_time), + to_date(fl_time), + date_add(fl_time, 2), + date_sub(fl_time, 2), + datediff(fl_time, "2000-01-01") +FROM date_udf_flight_orc; + +EXPLAIN SELECT + to_unix_timestamp(fl_date), + year(fl_date), + month(fl_date), + day(fl_date), + dayofmonth(fl_date), + weekofyear(fl_date), + date(fl_date), + to_date(fl_date), + date_add(fl_date, 2), + date_sub(fl_date, 2), + datediff(fl_date, "2000-01-01") +FROM date_udf_flight_orc; + +SELECT + to_unix_timestamp(fl_date), + year(fl_date), + month(fl_date), + day(fl_date), + dayofmonth(fl_date), + weekofyear(fl_date), + date(fl_date), + to_date(fl_date), + date_add(fl_date, 2), + date_sub(fl_date, 2), + datediff(fl_date, "2000-01-01") +FROM date_udf_flight_orc; + +EXPLAIN SELECT + year(fl_time) = year(fl_date), + month(fl_time) = month(fl_date), + day(fl_time) = day(fl_date), + dayofmonth(fl_time) = dayofmonth(fl_date), + weekofyear(fl_time) = weekofyear(fl_date), + date(fl_time) = date(fl_date), + to_date(fl_time) = to_date(fl_date), + date_add(fl_time, 2) = date_add(fl_date, 2), + date_sub(fl_time, 2) = date_sub(fl_date, 2), + datediff(fl_time, "2000-01-01") = datediff(fl_date, "2000-01-01") +FROM date_udf_flight_orc; + +-- Should all be true or NULL +SELECT + year(fl_time) = year(fl_date), + month(fl_time) = month(fl_date), + day(fl_time) = day(fl_date), + dayofmonth(fl_time) = dayofmonth(fl_date), + weekofyear(fl_time) = weekofyear(fl_date), + date(fl_time) = date(fl_date), + to_date(fl_time) = to_date(fl_date), + date_add(fl_time, 2) = date_add(fl_date, 2), + date_sub(fl_time, 2) = date_sub(fl_date, 2), + datediff(fl_time, "2000-01-01") = datediff(fl_date, "2000-01-01") +FROM date_udf_flight_orc; + +EXPLAIN SELECT + fl_date, + to_date(date_add(fl_date, 2)), + to_date(date_sub(fl_date, 2)), + datediff(fl_date, date_add(fl_date, 2)), + datediff(fl_date, date_sub(fl_date, 2)), + datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) +FROM date_udf_flight_orc LIMIT 10; + +SELECT + fl_date, + to_date(date_add(fl_date, 2)), + to_date(date_sub(fl_date, 2)), + datediff(fl_date, date_add(fl_date, 2)), + datediff(fl_date, date_sub(fl_date, 2)), + datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) +FROM date_udf_flight_orc LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q new file mode 100644 index 0000000000000..6e622007e5dcb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q @@ -0,0 +1,12 @@ +SET hive.vectorized.execution.enabled=true; + +SET hive.map.groupby.sorted=true; + +create table dtest(a int, b int) clustered by (a) sorted by (a) into 1 buckets stored as orc; +insert into table dtest select c,b from (select array(300,300,300,300,300) as a, 1 as b from src limit 1) y lateral view explode(a) t1 as c; + +explain select sum(distinct a), count(distinct a) from dtest; +select sum(distinct a), count(distinct a) from dtest; + +explain select sum(distinct cint), count(distinct cint), avg(distinct cint), std(distinct cint) from alltypesorc; +select sum(distinct cint), count(distinct cint), avg(distinct cint), std(distinct cint) from alltypesorc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q new file mode 100644 index 0000000000000..f390c2caafeb2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q @@ -0,0 +1,12 @@ +SET hive.vectorized.execution.enabled=true; +SET hive.auto.convert.join=true; +SET hive.auto.convert.join.nonconditionaltask=true; +SET hive.auto.convert.join.nonconditionaltask.size=1000000000; + +EXPLAIN SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint) + FROM alltypesorc t1 + JOIN alltypesorc t2 ON t1.cint = t2.cint; + +SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint) + FROM alltypesorc t1 + JOIN alltypesorc t2 ON t1.cint = t2.cint; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q new file mode 100644 index 0000000000000..d6b082467938f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q @@ -0,0 +1,107 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test math functions in vectorized mode to verify they run correctly end-to-end. + +explain +select + cdouble + ,Round(cdouble, 2) + ,Floor(cdouble) + ,Ceil(cdouble) + ,Rand() + ,Rand(98007) + ,Exp(ln(cdouble)) + ,Ln(cdouble) + ,Ln(cfloat) + ,Log10(cdouble) + -- Use log2 as a representative function to test all input types. + ,Log2(cdouble) + -- Use 15601.0 to test zero handling, as there are no zeroes in the table + ,Log2(cdouble - 15601.0) + ,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) + -- Test nesting + ,cos(-sin(log(cdouble)) + 3.14159) +from alltypesorc +-- limit output to a reasonably small number of rows +where cbigint % 500 = 0 +-- test use of a math function in the WHERE clause +and sin(cfloat) >= -1.0; + +select + cdouble + ,Round(cdouble, 2) + ,Floor(cdouble) + ,Ceil(cdouble) + -- Omit rand() from runtime test because it's nondeterministic. + -- ,Rand() + ,Rand(98007) + ,Exp(ln(cdouble)) + ,Ln(cdouble) + ,Ln(cfloat) + ,Log10(cdouble) + -- Use log2 as a representative function to test all input types. + ,Log2(cdouble) + -- Use 15601.0 to test zero handling, as there are no zeroes in the table + ,Log2(cdouble - 15601.0) + ,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) + -- Test nesting + ,cos(-sin(log(cdouble)) + 3.14159) +from alltypesorc +-- limit output to a reasonably small number of rows +where cbigint % 500 = 0 +-- test use of a math function in the WHERE clause +and sin(cfloat) >= -1.0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q new file mode 100644 index 0000000000000..ce4227cf0a700 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q @@ -0,0 +1,8 @@ +SET hive.vectorized.execution.enabled=true; +SET hive.auto.convert.join=true; +SET hive.auto.convert.join.nonconditionaltask=true; +SET hive.auto.convert.join.nonconditionaltask.size=1000000000; + +explain select sum(t1.td) from (select v1.csmallint as tsi, v1.cdouble as td from alltypesorc v1, alltypesorc v2 where v1.ctinyint=v2.ctinyint) t1 join alltypesorc v3 on t1.tsi=v3.csmallint; + +select sum(t1.td) from (select v1.csmallint as tsi, v1.cdouble as td from alltypesorc v1, alltypesorc v2 where v1.ctinyint=v2.ctinyint) t1 join alltypesorc v3 on t1.tsi=v3.csmallint; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q new file mode 100644 index 0000000000000..488d2f38859b3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q @@ -0,0 +1,18 @@ +--This query must pass even when vectorized reader is not available for +--RC files. The query must fall back to the non-vector mode and run successfully. + +CREATE table columnTable (key STRING, value STRING) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' +STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat'; + +FROM src +INSERT OVERWRITE TABLE columnTable SELECT src.key, src.value LIMIT 10; +describe columnTable; + +SET hive.vectorized.execution.enabled=true; + +SELECT key, value FROM columnTable ORDER BY key; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q new file mode 100644 index 0000000000000..6b60aa08c581b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q @@ -0,0 +1,10 @@ +SET hive.vectorized.execution.enabled=true; +SET hive.auto.convert.join=false; + +EXPLAIN SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint) + FROM alltypesorc t1 + JOIN alltypesorc t2 ON t1.cint = t2.cint; + +SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint) + FROM alltypesorc t1 + JOIN alltypesorc t2 ON t1.cint = t2.cint; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q new file mode 100644 index 0000000000000..96fe53da1ea35 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q @@ -0,0 +1,46 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test string functions in vectorized mode to verify end-to-end functionality. + +explain +select + substr(cstring1, 1, 2) + ,substr(cstring1, 2) + ,lower(cstring1) + ,upper(cstring1) + ,ucase(cstring1) + ,length(cstring1) + ,trim(cstring1) + ,ltrim(cstring1) + ,rtrim(cstring1) + ,concat(cstring1, cstring2) + ,concat('>', cstring1) + ,concat(cstring1, '<') + ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2)) +from alltypesorc +-- Limit the number of rows of output to a reasonable amount. +where cbigint % 237 = 0 +-- Test function use in the WHERE clause. +and length(substr(cstring1, 1, 2)) <= 2 +and cstring1 like '%'; + +select + substr(cstring1, 1, 2) + ,substr(cstring1, 2) + ,lower(cstring1) + ,upper(cstring1) + ,ucase(cstring1) + ,length(cstring1) + ,trim(cstring1) + ,ltrim(cstring1) + ,rtrim(cstring1) + ,concat(cstring1, cstring2) + ,concat('>', cstring1) + ,concat(cstring1, '<') + ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2)) +from alltypesorc +-- Limit the number of rows of output to a reasonable amount. +where cbigint % 237 = 0 +-- Test function use in the WHERE clause. +and length(substr(cstring1, 1, 2)) <= 2 +and cstring1 like '%'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q new file mode 100644 index 0000000000000..95eedd3b581d6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q @@ -0,0 +1,124 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test timestamp functions in vectorized mode to verify they run correctly end-to-end. + +CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC; + +INSERT OVERWRITE TABLE alltypesorc_string +SELECT + to_utc_timestamp(ctimestamp1, 'America/Los_Angeles'), + CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) +FROM alltypesorc +LIMIT 40; + +CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS ORC; + +INSERT INTO TABLE alltypesorc_wrong SELECT 'abcd' FROM alltypesorc LIMIT 1; +INSERT INTO TABLE alltypesorc_wrong SELECT '2000:01:01 00-00-00' FROM alltypesorc LIMIT 1; +INSERT INTO TABLE alltypesorc_wrong SELECT '0000-00-00 99:99:99' FROM alltypesorc LIMIT 1; + +EXPLAIN SELECT + to_unix_timestamp(ctimestamp1) AS c1, + year(ctimestamp1), + month(ctimestamp1), + day(ctimestamp1), + dayofmonth(ctimestamp1), + weekofyear(ctimestamp1), + hour(ctimestamp1), + minute(ctimestamp1), + second(ctimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +SELECT + to_unix_timestamp(ctimestamp1) AS c1, + year(ctimestamp1), + month(ctimestamp1), + day(ctimestamp1), + dayofmonth(ctimestamp1), + weekofyear(ctimestamp1), + hour(ctimestamp1), + minute(ctimestamp1), + second(ctimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +EXPLAIN SELECT + to_unix_timestamp(stimestamp1) AS c1, + year(stimestamp1), + month(stimestamp1), + day(stimestamp1), + dayofmonth(stimestamp1), + weekofyear(stimestamp1), + hour(stimestamp1), + minute(stimestamp1), + second(stimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +SELECT + to_unix_timestamp(stimestamp1) AS c1, + year(stimestamp1), + month(stimestamp1), + day(stimestamp1), + dayofmonth(stimestamp1), + weekofyear(stimestamp1), + hour(stimestamp1), + minute(stimestamp1), + second(stimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +EXPLAIN SELECT + to_unix_timestamp(ctimestamp1) = to_unix_timestamp(stimestamp1) AS c1, + year(ctimestamp1) = year(stimestamp1), + month(ctimestamp1) = month(stimestamp1), + day(ctimestamp1) = day(stimestamp1), + dayofmonth(ctimestamp1) = dayofmonth(stimestamp1), + weekofyear(ctimestamp1) = weekofyear(stimestamp1), + hour(ctimestamp1) = hour(stimestamp1), + minute(ctimestamp1) = minute(stimestamp1), + second(ctimestamp1) = second(stimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +-- Should all be true or NULL +SELECT + to_unix_timestamp(ctimestamp1) = to_unix_timestamp(stimestamp1) AS c1, + year(ctimestamp1) = year(stimestamp1), + month(ctimestamp1) = month(stimestamp1), + day(ctimestamp1) = day(stimestamp1), + dayofmonth(ctimestamp1) = dayofmonth(stimestamp1), + weekofyear(ctimestamp1) = weekofyear(stimestamp1), + hour(ctimestamp1) = hour(stimestamp1), + minute(ctimestamp1) = minute(stimestamp1), + second(ctimestamp1) = second(stimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +-- Wrong format. Should all be NULL. +EXPLAIN SELECT + to_unix_timestamp(stimestamp1) AS c1, + year(stimestamp1), + month(stimestamp1), + day(stimestamp1), + dayofmonth(stimestamp1), + weekofyear(stimestamp1), + hour(stimestamp1), + minute(stimestamp1), + second(stimestamp1) +FROM alltypesorc_wrong +ORDER BY c1; + +SELECT + to_unix_timestamp(stimestamp1) AS c1, + year(stimestamp1), + month(stimestamp1), + day(stimestamp1), + dayofmonth(stimestamp1), + weekofyear(stimestamp1), + hour(stimestamp1), + minute(stimestamp1), + second(stimestamp1) +FROM alltypesorc_wrong +ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q index 4e3d0572921d6..bc193554f9182 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q @@ -4,13 +4,13 @@ USE db1; CREATE TABLE table1 (key STRING, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE table1; CREATE TABLE table2 (key STRING, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE table2; -- relative reference, no alias diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q index b0b078ec628b3..95517c3bcd367 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS atab; CREATE TABLE IF NOT EXISTS atab (ks_uid BIGINT, sr_uid STRING, sr_id STRING, tstamp STRING, m_id STRING, act STRING, at_sr_uid STRING, tstamp_type STRING, original_m_id STRING, original_tstamp STRING, registered_flag TINYINT, at_ks_uid BIGINT) PARTITIONED BY (dt STRING,nt STRING); -LOAD DATA LOCAL INPATH '../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130312', nt='tw'); -LOAD DATA LOCAL INPATH '../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130311', nt='tw'); +LOAD DATA LOCAL INPATH '../../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130312', nt='tw'); +LOAD DATA LOCAL INPATH '../../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130311', nt='tw'); DROP TABLE IF EXISTS mstab; CREATE TABLE mstab(ks_uid INT, csc INT) PARTITIONED BY (dt STRING); -LOAD DATA LOCAL INPATH '../data/files/v2.txt' INTO TABLE mstab PARTITION (dt='20130311'); +LOAD DATA LOCAL INPATH '../../data/files/v2.txt' INTO TABLE mstab PARTITION (dt='20130311'); DROP VIEW IF EXISTS aa_view_tw; CREATE VIEW aa_view_tw AS SELECT ks_uid, sr_id, act, at_ks_uid, at_sr_uid, from_unixtime(CAST(CAST( tstamp as BIGINT)/1000 AS BIGINT),'yyyyMMdd') AS act_date, from_unixtime(CAST(CAST( original_tstamp AS BIGINT)/1000 AS BIGINT),'yyyyMMdd') AS content_creation_date FROM atab WHERE dt='20130312' AND nt='tw' AND ks_uid != at_ks_uid; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q index a7297db6104c0..2f22145518c56 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. testWindowing select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q index 9c7625dcd786a..67cab9f7b273a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; set hive.join.cache.size=1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q index 7c4ab386145d5..24f9ff73a30a1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. testQueryLevelPartitionColsNotInSelect select p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q index 2c3339095f22b..7e27c6b1c098d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; drop table over10k; @@ -27,12 +27,12 @@ create table over10k( bo boolean, s string, ts timestamp, - dec decimal, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select p_mfgr, p_retailprice, p_size, round(sum(p_retailprice) over w1 , 2) = round(sum(lag(p_retailprice,1,0.0)) over w1 + last_value(p_retailprice) over w1 , 2), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q index bb371e900975b..1c6e1aac37a1a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q @@ -10,12 +10,12 @@ create table over10k( bo boolean, s string, ts timestamp, - dec decimal, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, rank() over (partition by s order by si), sum(b) over (partition by s order by si) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q index 8a9d0012593e6..05da2ba7efeaf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q @@ -9,13 +9,13 @@ create table over10k( d double, bo boolean, s string, - ts timestamp, - dec decimal, + ts timestamp, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, row_number() over (partition by d order by dec) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q index 505c259f4b7b3..73e8192ee6e4c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q @@ -10,12 +10,12 @@ create table over10k( bo boolean, s string, ts timestamp, - dec decimal, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select i, ntile(10) over (partition by s order by i) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q index bf76867813e4e..4b951179e09e6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q @@ -10,12 +10,12 @@ create table over10k( bo boolean, s string, ts timestamp, - dec decimal, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, rank() over (partition by f order by t) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q index f22b992cd4386..0173ab7a3ac56 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q @@ -15,7 +15,7 @@ create table over10k( row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, min(i) over (partition by s) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q new file mode 100644 index 0000000000000..b813657baee47 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q @@ -0,0 +1,4 @@ +-- user-added aggregates should be usable as windowing functions +create temporary function mysum as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum'; + +select sum(key) over (), mysum(key) over () from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q index 7cc1367306956..6d8ce670454d0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q @@ -15,7 +15,7 @@ create table over10k( row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, sum(b) over (partition by i order by s,b rows unbounded preceding) from over10k limit 100; 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 aaefe84ce81ea..a90fc023e67d8 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 @@ -21,11 +21,9 @@ import org.scalatest.BeforeAndAfterAll import scala.reflect.ClassTag - import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.NativeCommand import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} -import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala index 46b11b582b26d..ca78dfba4fa38 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala @@ -22,7 +22,7 @@ import scala.util.Try import org.scalatest.FunSuite import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.sql.api.java.JavaSchemaRDD +import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD} import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.hive.test.TestHive @@ -33,9 +33,7 @@ class JavaHiveQLSuite extends FunSuite { lazy val javaCtx = new JavaSparkContext(TestHive.sparkContext) // There is a little trickery here to avoid instantiating two HiveContexts in the same JVM - lazy val javaHiveCtx = new JavaHiveContext(javaCtx) { - override val sqlContext = TestHive - } + lazy val javaHiveCtx = new JavaHiveContext(TestHive) test("SELECT * FROM src") { assert( 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 79cc7a3fcc7d6..44eb4cfa59335 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 @@ -142,14 +142,25 @@ abstract class HiveComparisonTest // Hack: Hive simply prints the result of a SET command to screen, // and does not return it as a query answer. case _: SetCommand => Seq("0") + case LogicalNativeCommand(c) if c.toLowerCase.contains("desc") => + answer + .filterNot(nonDeterministicLine) + .map(_.replaceAll("from deserializer", "")) + .map(_.replaceAll("None", "")) + .map(_.trim) + .filterNot(_ == "") case _: LogicalNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _: ExplainCommand => answer case _: DescribeCommand => // Filter out non-deterministic lines and lines which do not have actual results but // can introduce problems because of the way Hive formats these lines. // Then, remove empty lines. Do not sort the results. - answer.filterNot( - r => nonDeterministicLine(r) || ignoredLine(r)).map(_.trim).filterNot(_ == "") + answer + .filterNot(r => nonDeterministicLine(r) || ignoredLine(r)) + .map(_.replaceAll("from deserializer", "")) + .map(_.replaceAll("None", "")) + .map(_.trim) + .filterNot(_ == "") case plan => if (isSorted(plan)) answer else answer.sorted } orderedAnswer.map(cleanPaths) @@ -164,6 +175,7 @@ abstract class HiveComparisonTest "last_modified_by", "last_modified_time", "Owner:", + "COLUMN_STATS_ACCURATE", // The following are hive specific schema parameters which we do not need to match exactly. "numFiles", "numRows", @@ -237,6 +249,7 @@ abstract class HiveComparisonTest // the system to return the wrong answer. Since we have no intention of mirroring their // previously broken behavior we simply filter out changes to this setting. .filterNot(_ contains "hive.outerjoin.supports.filters") + .filterNot(_ contains "hive.exec.post.hooks") if (allQueries != queryList) logWarning(s"Simplifications made on unsupported operations for test $testCaseName") @@ -345,7 +358,7 @@ abstract class HiveComparisonTest (queryList, hiveResults, catalystResults).zipped.foreach { case (query, hive, (hiveQuery, catalyst)) => // Check that the results match unless its an EXPLAIN query. - val preparedHive = prepareAnswer(hiveQuery,hive) + val preparedHive = prepareAnswer(hiveQuery, hive) if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { 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 5de20175d9f57..322a25bb20837 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 @@ -508,19 +508,19 @@ class HiveQuerySuite extends HiveComparisonTest { // Describe a partition is a native command assertResult( Array( - Array("key", "int", HiveShim.getEmptyCommentsFieldValue), - Array("value", "string", HiveShim.getEmptyCommentsFieldValue), - Array("dt", "string", HiveShim.getEmptyCommentsFieldValue), - Array("", "", ""), - Array("# Partition Information", "", ""), + Array("key", "int"), + Array("value", "string"), + Array("dt", "string"), + Array(""), + Array("# Partition Information"), Array("# col_name", "data_type", "comment"), - Array("", "", ""), - Array("dt", "string", HiveShim.getEmptyCommentsFieldValue)) + Array(""), + Array("dt", "string")) ) { sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") .select('result) .collect() - .map(_.getString(0).split("\t").map(_.trim)) + .map(_.getString(0).replaceAll("None", "").trim.split("\t").map(_.trim)) } // Describe a registered temporary table. diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala deleted file mode 100644 index 6dde636965afd..0000000000000 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.net.URI -import java.util.{ArrayList => JArrayList} -import java.util.Properties -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} -import org.apache.hadoop.hive.ql.processors._ -import org.apache.hadoop.hive.ql.stats.StatsSetupConst -import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} -import org.apache.hadoop.{io => hadoopIo} -import org.apache.hadoop.mapred.InputFormat -import scala.collection.JavaConversions._ -import scala.language.implicitConversions - -/** - * A compatibility layer for interacting with Hive version 0.12.0. - */ -private[hive] object HiveShim { - val version = "0.12.0" - val metastoreDecimal = "decimal" - - def getTableDesc( - serdeClass: Class[_ <: Deserializer], - inputFormatClass: Class[_ <: InputFormat[_, _]], - outputFormatClass: Class[_], - properties: Properties) = { - new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) - } - - def createDriverResultsArray = new JArrayList[String] - - def processResults(results: JArrayList[String]) = results - - def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE - - def createDefaultDBIfNeeded(context: HiveContext) = { } - - /** The string used to denote an empty comments field in the schema. */ - def getEmptyCommentsFieldValue = "None" - - def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { - CommandProcessorFactory.get(cmd(0), conf) - } - - def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { - new HiveDecimal(bd) - } - - def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { - ColumnProjectionUtils.appendReadColumnIDs(conf, ids) - ColumnProjectionUtils.appendReadColumnNames(conf, names) - } - - def getExternalTmpPath(context: Context, uri: URI) = { - context.getExternalTmpFileURI(uri) - } - - def getDataLocationPath(p: Partition) = p.getPartitionPath - - def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) - -} - -class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) - extends FileSinkDesc(dir, tableInfo, compressed) { -} diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala new file mode 100644 index 0000000000000..2317d2e76341f --- /dev/null +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.net.URI +import java.util.{ArrayList => JArrayList} +import java.util.Properties +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.processors._ +import org.apache.hadoop.hive.ql.stats.StatsSetupConst +import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} +import org.apache.hadoop.{io => hadoopIo} +import org.apache.hadoop.mapred.InputFormat +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +/** + * A compatibility layer for interacting with Hive version 0.12.0. + */ +private[hive] object HiveShim { + val version = "0.12.0" + val metastoreDecimal = "decimal" + + def getTableDesc( + serdeClass: Class[_ <: Deserializer], + inputFormatClass: Class[_ <: InputFormat[_, _]], + outputFormatClass: Class[_], + properties: Properties) = { + new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) + } + + def createDriverResultsArray = new JArrayList[String] + + def processResults(results: JArrayList[String]) = results + + def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + + def createDefaultDBIfNeeded(context: HiveContext) = { } + + def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { + CommandProcessorFactory.get(cmd(0), conf) + } + + def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { + new HiveDecimal(bd) + } + + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + ColumnProjectionUtils.appendReadColumnIDs(conf, ids) + ColumnProjectionUtils.appendReadColumnNames(conf, names) + } + + def getExternalTmpPath(context: Context, uri: URI) = { + context.getExternalTmpFileURI(uri) + } + + def getDataLocationPath(p: Partition) = p.getPartitionPath + + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) + + def compatibilityBlackList = Seq( + "decimal_.*", + "drop_partitions_filter2", + "show_.*", + "serde_regex", + "udf_to_date", + "udaf_collect_set", + "udf_concat" + ) + +} + +class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) + extends FileSinkDesc(dir, tableInfo, compressed) { +} diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala deleted file mode 100644 index 8678c0c475db4..0000000000000 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala +++ /dev/null @@ -1,170 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.util.{ArrayList => JArrayList} -import java.util.Properties -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.common.`type`.{HiveDecimal} -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} -import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory -import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} -import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.Logging -import org.apache.hadoop.{io => hadoopIo} -import scala.collection.JavaConversions._ -import scala.language.implicitConversions - -/** - * A compatibility layer for interacting with Hive version 0.13.1. - */ -private[hive] object HiveShim { - val version = "0.13.1" - /* - * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded) - * Full support of new decimal feature need to be fixed in seperate PR. - */ - val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r - - def getTableDesc( - serdeClass: Class[_ <: Deserializer], - inputFormatClass: Class[_ <: InputFormat[_, _]], - outputFormatClass: Class[_], - properties: Properties) = { - new TableDesc(inputFormatClass, outputFormatClass, properties) - } - - def createDriverResultsArray = new JArrayList[Object] - - def processResults(results: JArrayList[Object]) = { - results.map { r => - r match { - case s: String => s - case a: Array[Object] => a(0).asInstanceOf[String] - } - } - } - - def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE - - def createDefaultDBIfNeeded(context: HiveContext) = { - context.runSqlHive("CREATE DATABASE default") - context.runSqlHive("USE default") - } - - /* The string used to denote an empty comments field in the schema. */ - def getEmptyCommentsFieldValue = "" - - def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { - CommandProcessorFactory.get(cmd, conf) - } - - def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { - HiveDecimal.create(bd) - } - - /* - * This function in hive-0.13 become private, but we have to do this to walkaround hive bug - */ - private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) { - val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "") - val result: StringBuilder = new StringBuilder(old) - var first: Boolean = old.isEmpty - - for (col <- cols) { - if (first) { - first = false - } else { - result.append(',') - } - result.append(col) - } - conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString) - } - - /* - * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty - */ - def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { - if (ids != null && ids.size > 0) { - ColumnProjectionUtils.appendReadColumns(conf, ids) - } - if (names != null && names.size > 0) { - appendReadColumnNames(conf, names) - } - } - - def getExternalTmpPath(context: Context, path: Path) = { - context.getExternalTmpPath(path.toUri) - } - - def getDataLocationPath(p: Partition) = p.getDataLocation - - def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsOf(tbl) - - /* - * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not. - * Fix it through wrapper. - * */ - implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = { - var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed) - f.setCompressCodec(w.compressCodec) - f.setCompressType(w.compressType) - f.setTableInfo(w.tableInfo) - f.setDestTableId(w.destTableId) - f - } -} - -/* - * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. - * Fix it through wrapper. - */ -class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) - extends Serializable with Logging { - var compressCodec: String = _ - var compressType: String = _ - var destTableId: Int = _ - - def setCompressed(compressed: Boolean) { - this.compressed = compressed - } - - def getDirName = dir - - def setDestTableId(destTableId: Int) { - this.destTableId = destTableId - } - - def setTableInfo(tableInfo: TableDesc) { - this.tableInfo = tableInfo - } - - def setCompressCodec(intermediateCompressorCodec: String) { - compressCodec = intermediateCompressorCodec - } - - def setCompressType(intermediateCompressType: String) { - compressType = intermediateCompressType - } -} diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala new file mode 100644 index 0000000000000..b8d893d8c1319 --- /dev/null +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.util.{ArrayList => JArrayList} +import java.util.Properties +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.StatsSetupConst +import org.apache.hadoop.hive.common.`type`.{HiveDecimal} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} +import org.apache.hadoop.mapred.InputFormat +import org.apache.spark.Logging +import org.apache.hadoop.{io => hadoopIo} +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +/** + * A compatibility layer for interacting with Hive version 0.13.1. + */ +private[hive] object HiveShim { + val version = "0.13.1" + /* + * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded) + * Full support of new decimal feature need to be fixed in seperate PR. + */ + val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r + + def getTableDesc( + serdeClass: Class[_ <: Deserializer], + inputFormatClass: Class[_ <: InputFormat[_, _]], + outputFormatClass: Class[_], + properties: Properties) = { + new TableDesc(inputFormatClass, outputFormatClass, properties) + } + + def createDriverResultsArray = new JArrayList[Object] + + def processResults(results: JArrayList[Object]) = { + results.map { r => + r match { + case s: String => s + case a: Array[Object] => a(0).asInstanceOf[String] + } + } + } + + def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + + def createDefaultDBIfNeeded(context: HiveContext) = { + context.runSqlHive("CREATE DATABASE default") + context.runSqlHive("USE default") + } + + def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { + CommandProcessorFactory.get(cmd, conf) + } + + def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { + HiveDecimal.create(bd) + } + + /* + * This function in hive-0.13 become private, but we have to do this to walkaround hive bug + */ + private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) { + val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "") + val result: StringBuilder = new StringBuilder(old) + var first: Boolean = old.isEmpty + + for (col <- cols) { + if (first) { + first = false + } else { + result.append(',') + } + result.append(col) + } + conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString) + } + + /* + * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty + */ + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + if (ids != null && ids.size > 0) { + ColumnProjectionUtils.appendReadColumns(conf, ids) + } + if (names != null && names.size > 0) { + appendReadColumnNames(conf, names) + } + } + + def getExternalTmpPath(context: Context, path: Path) = { + context.getExternalTmpPath(path.toUri) + } + + def getDataLocationPath(p: Partition) = p.getDataLocation + + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsOf(tbl) + + def compatibilityBlackList = Seq() + + /* + * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not. + * Fix it through wrapper. + * */ + implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = { + var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed) + f.setCompressCodec(w.compressCodec) + f.setCompressType(w.compressType) + f.setTableInfo(w.tableInfo) + f.setDestTableId(w.destTableId) + f + } +} + +/* + * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. + * Fix it through wrapper. + */ +class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) + extends Serializable with Logging { + var compressCodec: String = _ + var compressType: String = _ + var destTableId: Int = _ + + def setCompressed(compressed: Boolean) { + this.compressed = compressed + } + + def getDirName = dir + + def setDestTableId(destTableId: Int) { + this.destTableId = destTableId + } + + def setTableInfo(tableInfo: TableDesc) { + this.tableInfo = tableInfo + } + + def setCompressCodec(intermediateCompressorCodec: String) { + compressCodec = intermediateCompressorCodec + } + + def setCompressType(intermediateCompressType: String) { + compressType = intermediateCompressType + } +} From 9530316887612dca060a128fca34dd5a6ab2a9a9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 25 Oct 2014 00:06:57 -0700 Subject: [PATCH 020/115] [SPARK-2321] Stable pull-based progress / status API This pull request is a first step towards the implementation of a stable, pull-based progress / status API for Spark (see [SPARK-2321](https://issues.apache.org/jira/browse/SPARK-2321)). For now, I'd like to discuss the basic implementation, API names, and overall interface design. Once we arrive at a good design, I'll go back and add additional methods to expose more information via these API. #### Design goals: - Pull-based API - Usable from Java / Scala / Python (eventually, likely with a wrapper) - Can be extended to expose more information without introducing binary incompatibilities. - Returns immutable objects. - Don't leak any implementation details, preserving our freedom to change the implementation. #### Implementation: - Add public methods (`getJobInfo`, `getStageInfo`) to SparkContext to allow status / progress information to be retrieved. - Add public interfaces (`SparkJobInfo`, `SparkStageInfo`) for our API return values. These interfaces consist entirely of Java-style getter methods. The interfaces are currently implemented in Java. I decided to explicitly separate the interface from its implementation (`SparkJobInfoImpl`, `SparkStageInfoImpl`) in order to prevent users from constructing these responses themselves. -Allow an existing JobProgressListener to be used when constructing a live SparkUI. This allows us to re-use this listeners in the implementation of this status API. There are a few reasons why this listener re-use makes sense: - The status API and web UI are guaranteed to show consistent information. - These listeners are already well-tested. - The same garbage-collection / information retention configurations can apply to both this API and the web UI. - Extend JobProgressListener to maintain `jobId -> Job` and `stageId -> Stage` mappings. The progress API methods are implemented in a separate trait that's mixed into SparkContext. This helps to avoid SparkContext.scala from becoming larger and more difficult to read. Author: Josh Rosen Author: Josh Rosen Closes #2696 from JoshRosen/progress-reporting-api and squashes the following commits: e6aa78d [Josh Rosen] Add tests. b585c16 [Josh Rosen] Accept SparkListenerBus instead of more specific subclasses. c96402d [Josh Rosen] Address review comments. 2707f98 [Josh Rosen] Expose current stage attempt id c28ba76 [Josh Rosen] Update demo code: 646ff1d [Josh Rosen] Document spark.ui.retainedJobs. 7f47d6d [Josh Rosen] Clean up SparkUI constructors, per Andrew's feedback. b77b3d8 [Josh Rosen] Merge remote-tracking branch 'origin/master' into progress-reporting-api 787444c [Josh Rosen] Move status API methods into trait that can be mixed into SparkContext. f9a9a00 [Josh Rosen] More review comments: 3dc79af [Josh Rosen] Remove creation of unused listeners in SparkContext. 249ca16 [Josh Rosen] Address several review comments: da5648e [Josh Rosen] Add example of basic progress reporting in Java. 7319ffd [Josh Rosen] Add getJobIdsForGroup() and num*Tasks() methods. cc568e5 [Josh Rosen] Add note explaining that interfaces should not be implemented outside of Spark. 6e840d4 [Josh Rosen] Remove getter-style names and "consistent snapshot" semantics: 08cbec9 [Josh Rosen] Begin to sketch the interfaces for a stable, public status API. ac2d13a [Josh Rosen] Add jobId->stage, stageId->stage mappings in JobProgressListener 24de263 [Josh Rosen] Create UI listeners in SparkContext instead of in Tabs: --- .../org/apache/spark/JobExecutionStatus.java | 25 +++ .../java/org/apache/spark/SparkJobInfo.java | 30 ++++ .../java/org/apache/spark/SparkStageInfo.java | 34 +++++ .../scala/org/apache/spark/SparkContext.scala | 76 ++-------- .../org/apache/spark/SparkStatusAPI.scala | 142 ++++++++++++++++++ .../org/apache/spark/StatusAPIImpl.scala | 34 +++++ .../spark/api/java/JavaSparkContext.scala | 19 +++ .../deploy/history/FsHistoryProvider.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 4 +- .../scala/org/apache/spark/ui/SparkUI.scala | 108 ++++++++----- .../apache/spark/ui/env/EnvironmentTab.scala | 4 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 3 +- .../spark/ui/jobs/JobProgressListener.scala | 49 +++++- .../spark/ui/jobs/JobProgressPage.scala | 9 +- .../apache/spark/ui/jobs/JobProgressTab.scala | 10 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/UIData.scala | 8 + .../apache/spark/ui/storage/StorageTab.scala | 3 +- .../org/apache/spark/StatusAPISuite.scala | 78 ++++++++++ docs/configuration.md | 11 +- .../spark/examples/JavaStatusAPIDemo.java | 70 +++++++++ 21 files changed, 588 insertions(+), 134 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/JobExecutionStatus.java create mode 100644 core/src/main/java/org/apache/spark/SparkJobInfo.java create mode 100644 core/src/main/java/org/apache/spark/SparkStageInfo.java create mode 100644 core/src/main/scala/org/apache/spark/SparkStatusAPI.scala create mode 100644 core/src/main/scala/org/apache/spark/StatusAPIImpl.scala create mode 100644 core/src/test/scala/org/apache/spark/StatusAPISuite.scala create mode 100644 examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java diff --git a/core/src/main/java/org/apache/spark/JobExecutionStatus.java b/core/src/main/java/org/apache/spark/JobExecutionStatus.java new file mode 100644 index 0000000000000..6e161313702bb --- /dev/null +++ b/core/src/main/java/org/apache/spark/JobExecutionStatus.java @@ -0,0 +1,25 @@ +/* + * 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; + +public enum JobExecutionStatus { + RUNNING, + SUCCEEDED, + FAILED, + UNKNOWN +} diff --git a/core/src/main/java/org/apache/spark/SparkJobInfo.java b/core/src/main/java/org/apache/spark/SparkJobInfo.java new file mode 100644 index 0000000000000..4e3c983b1170a --- /dev/null +++ b/core/src/main/java/org/apache/spark/SparkJobInfo.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark; + +/** + * Exposes information about Spark Jobs. + * + * This interface is not designed to be implemented outside of Spark. We may add additional methods + * which may break binary compatibility with outside implementations. + */ +public interface SparkJobInfo { + int jobId(); + int[] stageIds(); + JobExecutionStatus status(); +} diff --git a/core/src/main/java/org/apache/spark/SparkStageInfo.java b/core/src/main/java/org/apache/spark/SparkStageInfo.java new file mode 100644 index 0000000000000..04e2247210ecc --- /dev/null +++ b/core/src/main/java/org/apache/spark/SparkStageInfo.java @@ -0,0 +1,34 @@ +/* + * 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; + +/** + * Exposes information about Spark Stages. + * + * This interface is not designed to be implemented outside of Spark. We may add additional methods + * which may break binary compatibility with outside implementations. + */ +public interface SparkStageInfo { + int stageId(); + int currentAttemptId(); + String name(); + int numTasks(); + int numActiveTasks(); + int numCompletedTasks(); + int numFailedTasks(); +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4565832334420..e8fdfff04390d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, UUID} import java.util.UUID.randomUUID import scala.collection.{Map, Set} -import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} @@ -51,6 +50,7 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.JobProgressListener import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** @@ -61,7 +61,7 @@ import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, Metadat * this config overrides the default configs as well as system properties. */ -class SparkContext(config: SparkConf) extends Logging { +class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It @@ -224,10 +224,15 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - // Initialize the Spark UI, registering all associated listeners + + private[spark] val jobProgressListener = new JobProgressListener(conf) + listenerBus.addListener(jobProgressListener) + + // Initialize the Spark UI private[spark] val ui: Option[SparkUI] = if (conf.getBoolean("spark.ui.enabled", true)) { - Some(new SparkUI(this)) + Some(SparkUI.createLiveUI(this, conf, listenerBus, jobProgressListener, + env.securityManager,appName)) } else { // For tests, do not enable the UI None @@ -854,69 +859,6 @@ class SparkContext(config: SparkConf) extends Logging { /** The version of Spark on which this application is running. */ def version = SPARK_VERSION - /** - * Return a map from the slave to the max memory available for caching and the remaining - * memory available for caching. - */ - def getExecutorMemoryStatus: Map[String, (Long, Long)] = { - env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) => - (blockManagerId.host + ":" + blockManagerId.port, mem) - } - } - - /** - * :: DeveloperApi :: - * Return information about what RDDs are cached, if they are in mem or on disk, how much space - * they take, etc. - */ - @DeveloperApi - def getRDDStorageInfo: Array[RDDInfo] = { - val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray - StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) - rddInfos.filter(_.isCached) - } - - /** - * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call. - * Note that this does not necessarily mean the caching or computation was successful. - */ - def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - - /** - * :: DeveloperApi :: - * Return information about blocks stored in all of the slaves - */ - @DeveloperApi - def getExecutorStorageStatus: Array[StorageStatus] = { - env.blockManager.master.getStorageStatus - } - - /** - * :: DeveloperApi :: - * Return pools for fair scheduler - */ - @DeveloperApi - def getAllPools: Seq[Schedulable] = { - // TODO(xiajunluan): We should take nested pools into account - taskScheduler.rootPool.schedulableQueue.toSeq - } - - /** - * :: DeveloperApi :: - * Return the pool associated with the given name, if one exists - */ - @DeveloperApi - def getPoolForName(pool: String): Option[Schedulable] = { - Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool)) - } - - /** - * Return current scheduling mode - */ - def getSchedulingMode: SchedulingMode.SchedulingMode = { - taskScheduler.schedulingMode - } - /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. diff --git a/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala b/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala new file mode 100644 index 0000000000000..1982499c5e1d3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala @@ -0,0 +1,142 @@ +/* + * 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 scala.collection.Map +import scala.collection.JavaConversions._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.{SchedulingMode, Schedulable} +import org.apache.spark.storage.{StorageStatus, StorageUtils, RDDInfo} + +/** + * Trait that implements Spark's status APIs. This trait is designed to be mixed into + * SparkContext; it allows the status API code to live in its own file. + */ +private[spark] trait SparkStatusAPI { this: SparkContext => + + /** + * Return a map from the slave to the max memory available for caching and the remaining + * memory available for caching. + */ + def getExecutorMemoryStatus: Map[String, (Long, Long)] = { + env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) => + (blockManagerId.host + ":" + blockManagerId.port, mem) + } + } + + /** + * :: DeveloperApi :: + * Return information about what RDDs are cached, if they are in mem or on disk, how much space + * they take, etc. + */ + @DeveloperApi + def getRDDStorageInfo: Array[RDDInfo] = { + val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray + StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) + rddInfos.filter(_.isCached) + } + + /** + * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call. + * Note that this does not necessarily mean the caching or computation was successful. + */ + def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap + + /** + * :: DeveloperApi :: + * Return information about blocks stored in all of the slaves + */ + @DeveloperApi + def getExecutorStorageStatus: Array[StorageStatus] = { + env.blockManager.master.getStorageStatus + } + + /** + * :: DeveloperApi :: + * Return pools for fair scheduler + */ + @DeveloperApi + def getAllPools: Seq[Schedulable] = { + // TODO(xiajunluan): We should take nested pools into account + taskScheduler.rootPool.schedulableQueue.toSeq + } + + /** + * :: DeveloperApi :: + * Return the pool associated with the given name, if one exists + */ + @DeveloperApi + def getPoolForName(pool: String): Option[Schedulable] = { + Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool)) + } + + /** + * Return current scheduling mode + */ + def getSchedulingMode: SchedulingMode.SchedulingMode = { + taskScheduler.schedulingMode + } + + + /** + * Return a list of all known jobs in a particular job group. The returned list may contain + * running, failed, and completed jobs, and may vary across invocations of this method. This + * method does not guarantee the order of the elements in its result. + */ + def getJobIdsForGroup(jobGroup: String): Array[Int] = { + jobProgressListener.synchronized { + val jobData = jobProgressListener.jobIdToData.valuesIterator + jobData.filter(_.jobGroup.exists(_ == jobGroup)).map(_.jobId).toArray + } + } + + /** + * Returns job information, or `None` if the job info could not be found or was garbage collected. + */ + def getJobInfo(jobId: Int): Option[SparkJobInfo] = { + jobProgressListener.synchronized { + jobProgressListener.jobIdToData.get(jobId).map { data => + new SparkJobInfoImpl(jobId, data.stageIds.toArray, data.status) + } + } + } + + /** + * Returns stage information, or `None` if the stage info could not be found or was + * garbage collected. + */ + def getStageInfo(stageId: Int): Option[SparkStageInfo] = { + jobProgressListener.synchronized { + for ( + info <- jobProgressListener.stageIdToInfo.get(stageId); + data <- jobProgressListener.stageIdToData.get((stageId, info.attemptId)) + ) yield { + new SparkStageInfoImpl( + stageId, + info.attemptId, + info.name, + info.numTasks, + data.numActiveTasks, + data.numCompleteTasks, + data.numFailedTasks) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala new file mode 100644 index 0000000000000..90b47c847fbca --- /dev/null +++ b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala @@ -0,0 +1,34 @@ +/* + * 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 + +private class SparkJobInfoImpl ( + val jobId: Int, + val stageIds: Array[Int], + val status: JobExecutionStatus) + extends SparkJobInfo + +private class SparkStageInfoImpl( + val stageId: Int, + val currentAttemptId: Int, + val name: String, + val numTasks: Int, + val numActiveTasks: Int, + val numCompletedTasks: Int, + val numFailedTasks: Int) + extends SparkStageInfo diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 791d853a015a1..45168ba62d3c1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -132,6 +132,25 @@ class JavaSparkContext(val sc: SparkContext) /** Default min number of partitions for Hadoop RDDs when not given by user */ def defaultMinPartitions: java.lang.Integer = sc.defaultMinPartitions + + /** + * Return a list of all known jobs in a particular job group. The returned list may contain + * running, failed, and completed jobs, and may vary across invocations of this method. This + * method does not guarantee the order of the elements in its result. + */ + def getJobIdsForGroup(jobGroup: String): Array[Int] = sc.getJobIdsForGroup(jobGroup) + + /** + * Returns job information, or `null` if the job info could not be found or was garbage collected. + */ + def getJobInfo(jobId: Int): SparkJobInfo = sc.getJobInfo(jobId).orNull + + /** + * Returns stage information, or `null` if the stage info could not be found or was + * garbage collected. + */ + def getStageInfo(stageId: Int): SparkStageInfo = sc.getStageInfo(stageId).orNull + /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag 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 481f6c93c6a8d..2d1609b973607 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 @@ -112,7 +112,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val ui = { val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) - new SparkUI(conf, appSecManager, replayBus, appId, + SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, s"${HistoryServer.UI_PATH_PREFIX}/$appId") // Do not call ui.bind() to avoid creating a new server for each application } 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 3b6bb9fe128a4..2f81d472d7b78 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 @@ -721,8 +721,8 @@ private[spark] class Master( try { val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) - val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)", - HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), + appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}") replayBus.replay() appIdToUI(app.id) = ui webUi.attachSparkUI(ui) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index cccd59d122a92..049938f827291 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,47 +21,30 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.env.EnvironmentTab -import org.apache.spark.ui.exec.ExecutorsTab -import org.apache.spark.ui.jobs.JobProgressTab -import org.apache.spark.ui.storage.StorageTab +import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} +import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} +import org.apache.spark.ui.jobs.{JobProgressListener, JobProgressTab} +import org.apache.spark.ui.storage.{StorageListener, StorageTab} /** * Top level user interface for a Spark application. */ -private[spark] class SparkUI( - val sc: SparkContext, +private[spark] class SparkUI private ( + val sc: Option[SparkContext], val conf: SparkConf, val securityManager: SecurityManager, - val listenerBus: SparkListenerBus, + val environmentListener: EnvironmentListener, + val storageStatusListener: StorageStatusListener, + val executorsListener: ExecutorsListener, + val jobProgressListener: JobProgressListener, + val storageListener: StorageListener, var appName: String, - val basePath: String = "") + val basePath: String) extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") with Logging { - def this(sc: SparkContext) = this(sc, sc.conf, sc.env.securityManager, sc.listenerBus, sc.appName) - def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) - - def this( - conf: SparkConf, - securityManager: SecurityManager, - listenerBus: SparkListenerBus, - appName: String, - basePath: String) = - this(null, conf, securityManager, listenerBus, appName, basePath) - - // If SparkContext is not provided, assume the associated application is not live - val live = sc != null - - // Maintain executor storage status through Spark events - val storageStatusListener = new StorageStatusListener - - initialize() - /** Initialize all components of the server. */ def initialize() { - listenerBus.addListener(storageStatusListener) val jobProgressTab = new JobProgressTab(this) attachTab(jobProgressTab) attachTab(new StorageTab(this)) @@ -71,10 +54,10 @@ private[spark] class SparkUI( attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) attachHandler( createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) - if (live) { - sc.env.metricsSystem.getServletHandlers.foreach(attachHandler) - } + // If the UI is live, then serve + sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) } } + initialize() def getAppName = appName @@ -83,11 +66,6 @@ private[spark] class SparkUI( appName = name } - /** Register the given listener with the listener bus. */ - def registerListener(listener: SparkListener) { - listenerBus.addListener(listener) - } - /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() @@ -116,4 +94,60 @@ private[spark] object SparkUI { def getUIPort(conf: SparkConf): Int = { conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) } + + def createLiveUI( + sc: SparkContext, + conf: SparkConf, + listenerBus: SparkListenerBus, + jobProgressListener: JobProgressListener, + securityManager: SecurityManager, + appName: String): SparkUI = { + create(Some(sc), conf, listenerBus, securityManager, appName, + jobProgressListener = Some(jobProgressListener)) + } + + def createHistoryUI( + conf: SparkConf, + listenerBus: SparkListenerBus, + securityManager: SecurityManager, + appName: String, + basePath: String): SparkUI = { + create(None, conf, listenerBus, securityManager, appName, basePath) + } + + /** + * Create a new Spark UI. + * + * @param sc optional SparkContext; this can be None when reconstituting a UI from event logs. + * @param jobProgressListener if supplied, this JobProgressListener will be used; otherwise, the + * web UI will create and register its own JobProgressListener. + */ + private def create( + sc: Option[SparkContext], + conf: SparkConf, + listenerBus: SparkListenerBus, + securityManager: SecurityManager, + appName: String, + basePath: String = "", + jobProgressListener: Option[JobProgressListener] = None): SparkUI = { + + val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse { + val listener = new JobProgressListener(conf) + listenerBus.addListener(listener) + listener + } + + val environmentListener = new EnvironmentListener + val storageStatusListener = new StorageStatusListener + val executorsListener = new ExecutorsListener(storageStatusListener) + val storageListener = new StorageListener(storageStatusListener) + + listenerBus.addListener(environmentListener) + listenerBus.addListener(storageStatusListener) + listenerBus.addListener(executorsListener) + listenerBus.addListener(storageListener) + + new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener, + executorsListener, _jobProgressListener, storageListener, appName, basePath) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index 0d158fbe638d3..f62260c6f6e1d 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -22,10 +22,8 @@ import org.apache.spark.scheduler._ import org.apache.spark.ui._ private[ui] class EnvironmentTab(parent: SparkUI) extends SparkUITab(parent, "environment") { - val listener = new EnvironmentListener - + val listener = parent.environmentListener attachPage(new EnvironmentPage(this)) - parent.registerListener(listener) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 61eb111cd9100..689cf02b25b70 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -26,10 +26,9 @@ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.{SparkUI, SparkUITab} private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { - val listener = new ExecutorsListener(parent.storageStatusListener) + val listener = parent.executorsListener attachPage(new ExecutorsPage(this)) - parent.registerListener(listener) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index eaeb861f59e5a..b5207360510dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -40,17 +40,25 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { import JobProgressListener._ + type JobId = Int + type StageId = Int + type StageAttemptId = Int + // How many stages to remember val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) + // How many jobs to remember + val retailedJobs = conf.getInt("spark.ui.retainedJobs", DEFAULT_RETAINED_JOBS) - // Map from stageId to StageInfo - val activeStages = new HashMap[Int, StageInfo] - - // Map from (stageId, attemptId) to StageUIData - val stageIdToData = new HashMap[(Int, Int), StageUIData] + val activeJobs = new HashMap[JobId, JobUIData] + val completedJobs = ListBuffer[JobUIData]() + val failedJobs = ListBuffer[JobUIData]() + val jobIdToData = new HashMap[JobId, JobUIData] + val activeStages = new HashMap[StageId, StageInfo] val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() + val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData] + val stageIdToInfo = new HashMap[StageId, StageInfo] // Map from pool name to a hash map (map from stage id to StageInfo). val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() @@ -61,8 +69,32 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { def blockManagerIds = executorIdToBlockManagerId.values.toSeq + override def onJobStart(jobStart: SparkListenerJobStart) = synchronized { + val jobGroup = Option(jobStart.properties).map(_.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + val jobData: JobUIData = + new JobUIData(jobStart.jobId, jobStart.stageIds, jobGroup, JobExecutionStatus.RUNNING) + jobIdToData(jobStart.jobId) = jobData + activeJobs(jobStart.jobId) = jobData + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { + val jobData = activeJobs.remove(jobEnd.jobId).getOrElse { + logWarning(s"Job completed for unknown job ${jobEnd.jobId}") + new JobUIData(jobId = jobEnd.jobId) + } + jobEnd.jobResult match { + case JobSucceeded => + completedJobs += jobData + jobData.status = JobExecutionStatus.SUCCEEDED + case JobFailed(exception) => + failedJobs += jobData + jobData.status = JobExecutionStatus.FAILED + } + } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo + stageIdToInfo(stage.stageId) = stage val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), { logWarning("Stage completed for unknown stage " + stage.stageId) new StageUIData @@ -89,7 +121,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { val toRemove = math.max(retainedStages / 10, 1) - stages.take(toRemove).foreach { s => stageIdToData.remove((s.stageId, s.attemptId)) } + stages.take(toRemove).foreach { s => + stageIdToData.remove((s.stageId, s.attemptId)) + stageIdToInfo.remove(s.stageId) + } stages.trimStart(toRemove) } } @@ -103,6 +138,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) }.getOrElse(DEFAULT_POOL_NAME) + stageIdToInfo(stage.stageId) = stage val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), new StageUIData) stageData.schedulingPool = poolName @@ -277,4 +313,5 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { private object JobProgressListener { val DEFAULT_POOL_NAME = "default" val DEFAULT_RETAINED_STAGES = 1000 + val DEFAULT_RETAINED_JOBS = 1000 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 1e02f1225d344..6e718eecdd52a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -26,7 +26,6 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { - private val live = parent.live private val sc = parent.sc private val listener = parent.listener private def isFairScheduler = parent.isFairScheduler @@ -47,17 +46,17 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs - val pools = if (live) sc.getAllPools else Seq[Schedulable]() + val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) val poolTable = new PoolTable(pools, parent) val summary: NodeSeq =
    - {if (live) { + {if (sc.isDefined) { // Total duration is not meaningful unless the UI is live
  • Total Duration: - {UIUtils.formatDuration(now - sc.startTime)} + {UIUtils.formatDuration(now - sc.get.startTime)}
  • }}
  • @@ -80,7 +79,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("")
val content = summary ++ - {if (live && isFairScheduler) { + {if (sc.isDefined && isFairScheduler) {

{pools.size} Fair Scheduler Pools

++ poolTable.toNodeSeq } else { Seq[Node]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index c16542c9db30f..03ca918e2e8b3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -25,16 +25,14 @@ import org.apache.spark.ui.{SparkUI, SparkUITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "stages") { - val live = parent.live val sc = parent.sc - val conf = if (live) sc.conf else new SparkConf - val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) - val listener = new JobProgressListener(conf) + val conf = sc.map(_.conf).getOrElse(new SparkConf) + val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val listener = parent.jobProgressListener attachPage(new JobProgressPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) - parent.registerListener(listener) def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) @@ -43,7 +41,7 @@ private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "st val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { - sc.cancelStage(stageId) + sc.get.cancelStage(stageId) } // Do a quick pause here to give Spark time to kill the stage so it shows up as // killed after the refresh. Note that this will block the serving thread so the diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 7a6c7d1a497ed..770d99eea1c9d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -26,7 +26,6 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { - private val live = parent.live private val sc = parent.sc private val listener = parent.listener @@ -42,7 +41,7 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs - val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() + val pools = sc.map(_.getPoolForName(poolName).get).toSeq val poolTable = new PoolTable(pools, parent) val content = diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index a336bf7e1ed02..e2813f8eb5ab9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui.jobs +import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} import org.apache.spark.util.collection.OpenHashSet @@ -36,6 +37,13 @@ private[jobs] object UIData { var diskBytesSpilled : Long = 0 } + class JobUIData( + var jobId: Int = -1, + var stageIds: Seq[Int] = Seq.empty, + var jobGroup: Option[String] = None, + var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN + ) + class StageUIData { var numActiveTasks: Int = _ var numCompleteTasks: Int = _ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 76097f1c51f8e..a81291d505583 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -26,11 +26,10 @@ import org.apache.spark.storage._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { - val listener = new StorageListener(parent.storageStatusListener) + val listener = parent.storageListener attachPage(new StoragePage(this)) attachPage(new RDDPage(this)) - parent.registerListener(listener) } /** diff --git a/core/src/test/scala/org/apache/spark/StatusAPISuite.scala b/core/src/test/scala/org/apache/spark/StatusAPISuite.scala new file mode 100644 index 0000000000000..4468fba8c1dff --- /dev/null +++ b/core/src/test/scala/org/apache/spark/StatusAPISuite.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 + +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + +import org.scalatest.{Matchers, FunSuite} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.JobExecutionStatus._ +import org.apache.spark.SparkContext._ + +class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext { + + test("basic status API usage") { + val jobFuture = sc.parallelize(1 to 10000, 2).map(identity).groupBy(identity).collectAsync() + val jobId: Int = eventually(timeout(10 seconds)) { + val jobIds = jobFuture.jobIds + jobIds.size should be(1) + jobIds.head + } + val jobInfo = eventually(timeout(10 seconds)) { + sc.getJobInfo(jobId).get + } + jobInfo.status() should not be FAILED + val stageIds = jobInfo.stageIds() + stageIds.size should be(2) + + val firstStageInfo = eventually(timeout(10 seconds)) { + sc.getStageInfo(stageIds(0)).get + } + firstStageInfo.stageId() should be(stageIds(0)) + firstStageInfo.currentAttemptId() should be(0) + firstStageInfo.numTasks() should be(2) + eventually(timeout(10 seconds)) { + val updatedFirstStageInfo = sc.getStageInfo(stageIds(0)).get + updatedFirstStageInfo.numCompletedTasks() should be(2) + updatedFirstStageInfo.numActiveTasks() should be(0) + updatedFirstStageInfo.numFailedTasks() should be(0) + } + } + + test("getJobIdsForGroup()") { + sc.setJobGroup("my-job-group", "description") + sc.getJobIdsForGroup("my-job-group") should be (Seq.empty) + val firstJobFuture = sc.parallelize(1 to 1000).countAsync() + val firstJobId = eventually(timeout(10 seconds)) { + firstJobFuture.jobIds.head + } + eventually(timeout(10 seconds)) { + sc.getJobIdsForGroup("my-job-group") should be (Seq(firstJobId)) + } + val secondJobFuture = sc.parallelize(1 to 1000).countAsync() + val secondJobId = eventually(timeout(10 seconds)) { + secondJobFuture.jobIds.head + } + eventually(timeout(10 seconds)) { + sc.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId)) + } + } +} \ No newline at end of file diff --git a/docs/configuration.md b/docs/configuration.md index 66738d3ca754e..3007706a2586e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -375,7 +375,16 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedStages 1000 - How many stages the Spark UI remembers before garbage collecting. + How many stages the Spark UI and status APIs remember before garbage + collecting. + + + + spark.ui.retainedJobs + 1000 + + How many stages the Spark UI and status APIs remember before garbage + collecting. diff --git a/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java b/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java new file mode 100644 index 0000000000000..430e96ab14d9d --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java @@ -0,0 +1,70 @@ +/* + * 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; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkJobInfo; +import org.apache.spark.SparkStageInfo; +import org.apache.spark.api.java.JavaFutureAction; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +import java.util.Arrays; +import java.util.List; + +/** + * Example of using Spark's status APIs from Java. + */ +public final class JavaStatusAPIDemo { + + public static final String APP_NAME = "JavaStatusAPIDemo"; + + public static final class IdentityWithDelay implements Function { + @Override + public T call(T x) throws Exception { + Thread.sleep(2 * 1000); // 2 seconds + return x; + } + } + + public static void main(String[] args) throws Exception { + SparkConf sparkConf = new SparkConf().setAppName(APP_NAME); + final JavaSparkContext sc = new JavaSparkContext(sparkConf); + + // Example of implementing a progress reporter for a simple job. + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map( + new IdentityWithDelay()); + JavaFutureAction> jobFuture = rdd.collectAsync(); + while (!jobFuture.isDone()) { + Thread.sleep(1000); // 1 second + List jobIds = jobFuture.jobIds(); + if (jobIds.isEmpty()) { + continue; + } + int currentJobId = jobIds.get(jobIds.size() - 1); + SparkJobInfo jobInfo = sc.getJobInfo(currentJobId); + SparkStageInfo stageInfo = sc.getStageInfo(jobInfo.stageIds()[0]); + System.out.println(stageInfo.numTasks() + " tasks total: " + stageInfo.numActiveTasks() + + " active, " + stageInfo.numCompletedTasks() + " complete"); + } + + System.out.println("Job results are: " + jobFuture.get()); + sc.stop(); + } +} From e41786c77482d3f9e3c01cfd583c8899815c3106 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 25 Oct 2014 01:20:39 -0700 Subject: [PATCH 021/115] [SPARK-4088] [PySpark] Python worker should exit after socket is closed by JVM In case of take() or exception in Python, python worker may exit before JVM read() all the response, then the write thread may raise "Connection reset" exception. Python should always wait JVM to close the socket first. cc JoshRosen This is a warm fix, or the tests will be flaky, sorry for that. Author: Davies Liu Closes #2941 from davies/fix_exit and squashes the following commits: 9d4d21e [Davies Liu] fix race --- python/pyspark/daemon.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index dbb34775d9ac5..f09587f211708 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -62,8 +62,7 @@ def worker(sock): exit_code = compute_real_exit_code(exc.code) finally: outfile.flush() - if exit_code: - os._exit(exit_code) + return exit_code # Cleanup zombie children @@ -160,10 +159,13 @@ def handle_sigterm(*args): outfile.flush() outfile.close() while True: - worker(sock) - if not reuse: + code = worker(sock) + if not reuse or code: # wait for closing - while sock.recv(1024): + try: + while sock.recv(1024): + pass + except Exception: pass break gc.collect() From 2e52e4f815b3110912ecdb8ed18aa4b2209abcfb Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 25 Oct 2014 17:07:44 -0700 Subject: [PATCH 022/115] Revert "[SPARK-4056] Upgrade snappy-java to 1.1.1.5" This reverts commit 898b22ab1fe90e8a3935b19566465046f2256fa6. Reverting because this may be causing OOMs. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 030bea948b5ce..a1195262614dd 100644 --- a/pom.xml +++ b/pom.xml @@ -346,7 +346,7 @@ org.xerial.snappy snappy-java - 1.1.1.5 + 1.1.1.3 net.jpountz.lz4 From c6834440085b79f6d3e011f9e55ffd672be855fe Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 25 Oct 2014 20:07:44 -0700 Subject: [PATCH 023/115] [SPARK-4071] Unroll fails silently if BlockManager is small In tests, we may want to have BlockManagers of size < 1MB (spark.storage.unrollMemoryThreshold). However, these BlockManagers are useless because we can't unroll anything in them ever. At the very least we need to log a warning. tdas Author: Andrew Or Closes #2917 from andrewor14/unroll-safely-logging and squashes the following commits: 38947e3 [Andrew Or] Warn against starting a block manager that's too small fd621b4 [Andrew Or] Warn against failure to reserve initial memory threshold --- .../org/apache/spark/storage/MemoryStore.scala | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index edbc729c17ade..71305a46bf570 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -56,6 +56,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) (maxMemory * unrollFraction).toLong } + // Initial memory to request before unrolling any block + private val unrollMemoryThreshold: Long = + conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + + if (maxMemory < unrollMemoryThreshold) { + logWarning(s"Max memory ${Utils.bytesToString(maxMemory)} is less than the initial memory " + + s"threshold ${Utils.bytesToString(unrollMemoryThreshold)} needed to store a block in " + + s"memory. Please configure Spark with more memory.") + } + logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) /** Free memory not occupied by existing blocks. Note that this does not include unroll memory. */ @@ -213,7 +223,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Whether there is still enough memory for us to continue unrolling this block var keepUnrolling = true // Initial per-thread memory to request for unrolling blocks (bytes). Exposed for testing. - val initialMemoryThreshold = conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + val initialMemoryThreshold = unrollMemoryThreshold // How often to check whether we need to request more memory val memoryCheckPeriod = 16 // Memory currently reserved by this thread for this particular unrolling operation @@ -228,6 +238,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Request enough memory to begin unrolling keepUnrolling = reserveUnrollMemoryForThisThread(initialMemoryThreshold) + if (!keepUnrolling) { + logWarning(s"Failed to reserve initial memory threshold of " + + s"${Utils.bytesToString(initialMemoryThreshold)} for computing block $blockId in memory.") + } + // Unroll this block safely, checking whether we have exceeded our threshold periodically try { while (values.hasNext && keepUnrolling) { From df7974b8e59d00e8efbb61629418fa6265c1ddab Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 25 Oct 2014 23:18:02 -0700 Subject: [PATCH 024/115] SPARK-3359 [DOCS] sbt/sbt unidoc doesn't work with Java 8 This follows https://github.com/apache/spark/pull/2893 , but does not completely fix SPARK-3359 either. This fixes minor scaladoc/javadoc issues that Javadoc 8 will treat as errors. Author: Sean Owen Closes #2909 from srowen/SPARK-3359 and squashes the following commits: f62c347 [Sean Owen] Fix some javadoc issues that javadoc 8 considers errors. This is not all of the errors turned up when javadoc 8 runs on output of genjavadoc. --- core/src/main/java/org/apache/spark/TaskContext.java | 2 -- .../org/apache/spark/api/java/function/PairFunction.java | 3 ++- .../scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 6 +++--- .../scala/org/apache/spark/api/java/JavaPairRDD.scala | 4 ++-- .../org/apache/spark/api/java/JavaSparkContext.scala | 5 ++++- .../scala/org/apache/spark/mllib/feature/Normalizer.scala | 2 +- .../apache/spark/mllib/linalg/distributed/RowMatrix.scala | 7 ++++--- .../main/scala/org/apache/spark/mllib/util/MLUtils.scala | 8 ++++---- .../org/apache/spark/sql/api/java/JavaSchemaRDD.scala | 2 +- 9 files changed, 21 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java index 2d998d4c7a5d9..0d6973203eba1 100644 --- a/core/src/main/java/org/apache/spark/TaskContext.java +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -71,7 +71,6 @@ static void unset() { /** * Add a (Java friendly) listener to be executed on task completion. * This will be called in all situation - success, failure, or cancellation. - *

* An example use is for HadoopRDD to register a callback to close the input stream. */ public abstract TaskContext addTaskCompletionListener(TaskCompletionListener listener); @@ -79,7 +78,6 @@ static void unset() { /** * Add a listener in the form of a Scala closure to be executed on task completion. * This will be called in all situations - success, failure, or cancellation. - *

* An example use is for HadoopRDD to register a callback to close the input stream. */ public abstract TaskContext addTaskCompletionListener(final Function1 f); diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java index abd9bcc07ac61..99bf240a17225 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java @@ -22,7 +22,8 @@ import scala.Tuple2; /** - * A function that returns key-value pairs (Tuple2), and can be used to construct PairRDDs. + * A function that returns key-value pairs (Tuple2<K, V>), and can be used to + * construct PairRDDs. */ public interface PairFunction extends Serializable { public Tuple2 call(T t) throws Exception; diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index a6123bd108c11..8e8f7f6c4fda2 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -114,7 +114,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja * Return an RDD with the elements from `this` that are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be <= us. */ def subtract(other: JavaDoubleRDD): JavaDoubleRDD = fromRDD(srdd.subtract(other)) @@ -233,11 +233,11 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja * to the left except for the last which is closed * e.g. for the array * [1,10,20,50] the buckets are [1,10) [10,20) [20,50] - * e.g 1<=x<10 , 10<=x<20, 20<=x<50 + * e.g 1<=x<10 , 10<=x<20, 20<=x<50 * And on the input of 1 and 50 we would have a histogram of 1,0,0 * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched - * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets + * from an O(log n) insertion to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. * buckets array must be at least two elements diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index c38b96528d037..e37f3acaf6e30 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -392,7 +392,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return an RDD with the elements from `this` that are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be <= us. */ def subtract(other: JavaPairRDD[K, V]): JavaPairRDD[K, V] = fromRDD(rdd.subtract(other)) @@ -413,7 +413,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return an RDD with the pairs from `this` whose keys are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be <= us. */ def subtractByKey[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, V] = { implicit val ctag: ClassTag[W] = fakeClassTag diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 45168ba62d3c1..0565adf4d4ead 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -215,7 +215,10 @@ class JavaSparkContext(val sc: SparkContext) * hdfs://a-hdfs-path/part-nnnnn * }}} * - * Do `JavaPairRDD rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")`, + * Do + * {{{ + * JavaPairRDD rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path") + * }}} * *

then `rdd` contains * {{{ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index 4734251127bb4..dfad25d57c947 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} * :: Experimental :: * Normalizes samples individually to unit L^p^ norm * - * For any 1 <= p < Double.PositiveInfinity, normalizes samples using + * For any 1 <= p < Double.PositiveInfinity, normalizes samples using * sum(abs(vector).^p^)^(1/p)^ as norm. * * For p = Double.PositiveInfinity, max(abs(vector)) will be used as norm for normalization. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index ec2d481dccc22..10a515af88802 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -152,7 +152,7 @@ class RowMatrix( * storing the right singular vectors, is computed via matrix multiplication as * U = A * (V * S^-1^), if requested by user. The actual method to use is determined * automatically based on the cost: - * - If n is small (n < 100) or k is large compared with n (k > n / 2), we compute the Gramian + * - If n is small (n < 100) or k is large compared with n (k > n / 2), we compute the Gramian * matrix first and then compute its top eigenvalues and eigenvectors locally on the driver. * This requires a single pass with O(n^2^) storage on each executor and on the driver, and * O(n^2^ k) time on the driver. @@ -169,7 +169,8 @@ class RowMatrix( * @note The conditions that decide which method to use internally and the default parameters are * subject to change. * - * @param k number of leading singular values to keep (0 < k <= n). It might return less than k if + * @param k number of leading singular values to keep (0 < k <= n). + * It might return less than k if * there are numerically zero singular values or there are not enough Ritz values * converged before the maximum number of Arnoldi update iterations is reached (in case * that matrix A is ill-conditioned). @@ -192,7 +193,7 @@ class RowMatrix( /** * The actual SVD implementation, visible for testing. * - * @param k number of leading singular values to keep (0 < k <= n) + * @param k number of leading singular values to keep (0 < k <= n) * @param computeU whether to compute U * @param rCond the reciprocal condition number * @param maxIter max number of iterations (if ARPACK is used) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index ca35100aa99c6..dce0adffa6249 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -196,8 +196,8 @@ object MLUtils { /** * Load labeled data from a file. The data format used here is - * , ... - * where , are feature values in Double and is the corresponding label as Double. + * L, f1 f2 ... + * where f1, f2 are feature values in Double and L is the corresponding label as Double. * * @param sc SparkContext * @param dir Directory to the input data files. @@ -219,8 +219,8 @@ object MLUtils { /** * Save labeled data to a file. The data format used here is - * , ... - * where , are feature values in Double and is the corresponding label as Double. + * L, f1 f2 ... + * where f1, f2 are feature values in Double and L is the corresponding label as Double. * * @param data An RDD of LabeledPoints containing data to be saved. * @param dir Directory to save the data. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index e7faba0c7f620..1e0ccb368a276 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -193,7 +193,7 @@ class JavaSchemaRDD( * Return an RDD with the elements from `this` that are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be <= us. */ def subtract(other: JavaSchemaRDD): JavaSchemaRDD = this.baseSchemaRDD.subtract(other.baseSchemaRDD).toJavaSchemaRDD From b75954015fba8041ff25ab6d1a53b22b8f868526 Mon Sep 17 00:00:00 2001 From: Daniel Lemire Date: Sun, 26 Oct 2014 10:03:20 -0700 Subject: [PATCH 025/115] Update RoaringBitmap to 0.4.3 Roaring has been updated to version 0.4.3. We fixed a rarely occurring bug with serialization. No API or format changes were made. Author: Daniel Lemire Closes #2938 from lemire/master and squashes the following commits: 431f3a0 [Daniel Lemire] Recommended bug fix release --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a1195262614dd..2faf0c7dcffd3 100644 --- a/pom.xml +++ b/pom.xml @@ -445,7 +445,7 @@ org.roaringbitmap RoaringBitmap - 0.4.1 + 0.4.3 commons-net From bf589fc717c842d1998e3c3a523bc8775cb30269 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 26 Oct 2014 11:29:27 -0700 Subject: [PATCH 026/115] [SPARK-3616] Add basic Selenium tests to WebUISuite This patch adds Selenium tests for Spark's web UI. To avoid adding extra dependencies to the test environment, the tests use Selenium's HtmlUnitDriver, which is pure-Java, instead of, say, ChromeDriver. I added new tests to try to reproduce a few UI bugs reported on JIRA, namely SPARK-3021, SPARK-2105, and SPARK-2527. I wasn't able to reproduce these bugs; I suspect that the older ones might have been fixed by other patches. In order to use HtmlUnitDriver, I added an explicit dependency on the org.apache.httpcomponents version of httpclient in order to prevent jets3t's older version from taking precedence on the classpath. I also upgraded ScalaTest to 2.2.1. Author: Josh Rosen Author: Josh Rosen Closes #2474 from JoshRosen/webui-selenium-tests and squashes the following commits: fcc9e83 [Josh Rosen] scalautils -> scalactic package rename 510e54a [Josh Rosen] [SPARK-3616] Add basic Selenium tests to WebUISuite. --- core/pom.xml | 5 + .../scala/org/apache/spark/ui/UIUtils.scala | 7 +- .../scala/org/apache/spark/ui/WebUI.scala | 2 +- .../org/apache/spark/ui/storage/RDDPage.scala | 6 +- .../apache/spark/ui/storage/StoragePage.scala | 2 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 112 ++++++++++++++++++ pom.xml | 8 +- .../ExpressionEvaluationSuite.scala | 2 +- 8 files changed, 135 insertions(+), 9 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala diff --git a/core/pom.xml b/core/pom.xml index 7b68dbaea4789..320d1076f7c03 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -247,6 +247,11 @@ + + org.seleniumhq.selenium + selenium-java + test + org.scalatest scalatest_${scala.binary.version} 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 32e6b15bb0999..76714b1e6964f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.{Locale, Date} -import scala.xml.Node +import scala.xml.{Text, Node} import org.apache.spark.Logging @@ -239,7 +239,8 @@ private[spark] object UIUtils extends Logging { headers: Seq[String], generateDataRow: T => Seq[Node], data: Iterable[T], - fixedWidth: Boolean = false): Seq[Node] = { + fixedWidth: Boolean = false, + id: Option[String] = None): Seq[Node] = { var listingTableClass = TABLE_CLASS if (fixedWidth) { @@ -263,7 +264,7 @@ private[spark] object UIUtils extends Logging { } } } - +
{headerRow} {data.map(r => generateDataRow(r))} diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 5d88ca403a674..9be65a4a39a09 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -82,7 +82,7 @@ private[spark] abstract class WebUI( } /** Detach a handler from this UI. */ - def detachHandler(handler: ServletContextHandler) { + protected def detachHandler(handler: ServletContextHandler) { handlers -= handler serverInfo.foreach { info => info.rootHandler.removeHandler(handler) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 8a0075ae8daf7..12d23a92878cf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -39,7 +39,8 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { // Worker table val workers = storageStatusList.map((rddId, _)) - val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) + val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers, + id = Some("rdd-storage-by-worker-table")) // Block table val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) @@ -49,7 +50,8 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { .map { case (blockId, status) => (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) } - val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks) + val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks, + id = Some("rdd-storage-by-block-table")) val content =
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 83489ca0679ee..6ced6052d2b18 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -31,7 +31,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList - val content = UIUtils.listingTable(rddHeader, rddRow, rdds) + val content = UIUtils.listingTable(rddHeader, rddRow, rdds, id = Some("storage-by-rdd-table")) UIUtils.headerSparkPage("Storage", content, parent) } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala new file mode 100644 index 0000000000000..bacf6a16fc233 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -0,0 +1,112 @@ +/* + * 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 + +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.{SparkException, SparkConf, SparkContext} +import org.openqa.selenium.WebDriver +import org.openqa.selenium.htmlunit.HtmlUnitDriver +import org.scalatest._ +import org.scalatest.concurrent.Eventually._ +import org.scalatest.selenium.WebBrowser +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.LocalSparkContext._ + +/** + * Selenium tests for the Spark Web UI. These tests are not run by default + * because they're slow. + */ +@DoNotDiscover +class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { + implicit val webDriver: WebDriver = new HtmlUnitDriver + + /** + * Create a test SparkContext with the SparkUI enabled. + * It is safe to `get` the SparkUI directly from the SparkContext returned here. + */ + private def newSparkContext(): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + val sc = new SparkContext(conf) + assert(sc.ui.isDefined) + sc + } + + test("effects of unpersist() / persist() should be reflected") { + // Regression test for SPARK-2527 + withSpark(newSparkContext()) { sc => + val ui = sc.ui.get + val rdd = sc.parallelize(Seq(1, 2, 3)) + rdd.persist(StorageLevels.DISK_ONLY).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (ui.appUIAddress.stripSuffix("/") + "/storage") + val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq + tableRowText should contain (StorageLevels.DISK_ONLY.description) + } + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") + val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq + tableRowText should contain (StorageLevels.DISK_ONLY.description) + } + + rdd.unpersist() + rdd.persist(StorageLevels.MEMORY_ONLY).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (ui.appUIAddress.stripSuffix("/") + "/storage") + val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq + tableRowText should contain (StorageLevels.MEMORY_ONLY.description) + } + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") + val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq + tableRowText should contain (StorageLevels.MEMORY_ONLY.description) + } + } + } + + test("failed stages should not appear to be active") { + withSpark(newSparkContext()) { sc => + // Regression test for SPARK-3021 + intercept[SparkException] { + sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() + } + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to sc.ui.get.appUIAddress + find(id("active")).get.text should be("Active Stages (0)") + find(id("failed")).get.text should be("Failed Stages (1)") + } + + // Regression test for SPARK-2105 + class NotSerializable + val unserializableObject = new NotSerializable + intercept[SparkException] { + sc.parallelize(1 to 10).map { x => unserializableObject}.collect() + } + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to sc.ui.get.appUIAddress + find(id("active")).get.text should be("Active Stages (0)") + // The failure occurs before the stage becomes active, hence we should still show only one + // failed stage, not two: + find(id("failed")).get.text should be("Failed Stages (1)") + } + } + } +} diff --git a/pom.xml b/pom.xml index 2faf0c7dcffd3..2ebe1b8da588a 100644 --- a/pom.xml +++ b/pom.xml @@ -312,6 +312,12 @@ jsr305 1.3.9 + + org.seleniumhq.selenium + selenium-java + 2.42.2 + test + org.slf4j slf4j-api @@ -520,7 +526,7 @@ org.scalatest scalatest_${scala.binary.version} - 2.1.5 + 2.2.1 test diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 6dc5942023f9e..f134d73450515 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -23,7 +23,7 @@ import scala.collection.immutable.HashSet import org.scalatest.FunSuite import org.scalatest.Matchers._ -import org.scalautils.TripleEqualsSupport.Spread +import org.scalactic.TripleEqualsSupport.Spread import org.apache.spark.sql.catalyst.types._ From 677852c3fa734326f9642245584b05d5604963ac Mon Sep 17 00:00:00 2001 From: anant asthana Date: Sun, 26 Oct 2014 14:14:12 -0700 Subject: [PATCH 027/115] Just fixing comment that shows usage Author: anant asthana Closes #2948 from anantasty/patch-1 and squashes the following commits: d8fea0b [anant asthana] Just fixing comment that shows usage --- examples/src/main/python/streaming/hdfs_wordcount.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/streaming/hdfs_wordcount.py b/examples/src/main/python/streaming/hdfs_wordcount.py index 40faff0ccc7db..f7ffb5379681e 100644 --- a/examples/src/main/python/streaming/hdfs_wordcount.py +++ b/examples/src/main/python/streaming/hdfs_wordcount.py @@ -21,7 +21,7 @@ is the directory that Spark Streaming will use to find and read new text files. To run this on your local machine on directory `localdir`, run this example - $ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localdir + $ bin/spark-submit examples/src/main/python/streaming/hdfs_wordcount.py localdir Then create a text file in `localdir` and the words in the file will get counted. """ From 0af7e514c6790cb78ee216986c45dbfe1217cd7d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 26 Oct 2014 14:29:13 -0700 Subject: [PATCH 028/115] [SPARK-3925][SQL] Do not consider the ordering of qualifiers during comparison The orderings should not be considered during the comparison between old qualifiers and new qualifiers. Author: Liang-Chi Hsieh Closes #2783 from viirya/full_qualifier_comp and squashes the following commits: 89f652c [Liang-Chi Hsieh] modification for comment. abb5762 [Liang-Chi Hsieh] More comprehensive comparison of qualifiers. --- .../spark/sql/catalyst/expressions/namedExpressions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d023db44d8543..fe13a661f6f7a 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 @@ -156,7 +156,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea * Returns a copy of this [[AttributeReference]] with new qualifiers. */ override def withQualifiers(newQualifiers: Seq[String]) = { - if (newQualifiers == qualifiers) { + if (newQualifiers.toSet == qualifiers.toSet) { this } else { AttributeReference(name, dataType, nullable)(exprId, newQualifiers) From 879a16585808e8fe34bdede741565efc4c9f9bb3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 26 Oct 2014 15:24:39 -0700 Subject: [PATCH 029/115] [HOTFIX][SQL] Temporarily turn off hive-server tests. The thirift server is not available in the default (hive13) profile yet which is breaking all SQL only PRs. This turns off these test until #2685 is merged. Author: Michael Armbrust Closes #2950 from marmbrus/fixTests and squashes the following commits: 1a6dfee [Michael Armbrust] [HOTFIX][SQL] Temporarily turn of hive-server tests. --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index f55497ae2bfbd..972c8c8a21567 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -173,7 +173,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string #+ will be interpreted as a single test, which doesn't work. - SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test") + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test") else SBT_MAVEN_TEST_ARGS=("test") fi From 2838bf8aadd5228829c1a869863bc4da7877fdfb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 26 Oct 2014 16:10:09 -0700 Subject: [PATCH 030/115] [SPARK-3537][SPARK-3914][SQL] Refines in-memory columnar table statistics This PR refines in-memory columnar table statistics: 1. adds 2 more statistics for in-memory table columns: `count` and `sizeInBytes` 1. adds filter pushdown support for `IS NULL` and `IS NOT NULL`. 1. caches and propagates statistics in `InMemoryRelation` once the underlying cached RDD is materialized. Statistics are collected to driver side with an accumulator. This PR also fixes SPARK-3914 by properly propagating in-memory statistics. Author: Cheng Lian Closes #2860 from liancheng/propagates-in-mem-stats and squashes the following commits: 0cc5271 [Cheng Lian] Restricts visibility of o.a.s.s.c.p.l.Statistics c5ff904 [Cheng Lian] Fixes test table name conflict a8c818d [Cheng Lian] Refines tests 1d01074 [Cheng Lian] Bug fix: shouldn't call STRING.actualSize on null string value 7dc6a34 [Cheng Lian] Adds more in-memory table statistics and propagates them properly --- .../catalyst/expressions/AttributeMap.scala | 10 +- .../catalyst/plans/logical/LogicalPlan.scala | 31 +++-- .../spark/sql/columnar/ColumnStats.scala | 122 +++++++++--------- .../columnar/InMemoryColumnarTableScan.scala | 101 +++++++++------ .../spark/sql/execution/ExistingRDD.scala | 11 +- .../spark/sql/parquet/ParquetRelation.scala | 3 +- .../apache/spark/sql/CachedTableSuite.scala | 11 +- .../scala/org/apache/spark/sql/TestData.scala | 16 +-- .../spark/sql/columnar/ColumnStatsSuite.scala | 6 + .../columnar/PartitionBatchPruningSuite.scala | 76 ++++++----- .../spark/sql/execution/PlannerSuite.scala | 20 +++ 11 files changed, 240 insertions(+), 167 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala index 8364379644c90..82e760b6c6916 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -23,8 +23,7 @@ package org.apache.spark.sql.catalyst.expressions * of the name, or the expected nullability). */ object AttributeMap { - def apply[A](kvs: Seq[(Attribute, A)]) = - new AttributeMap(kvs.map(kv => (kv._1.exprId, (kv._1, kv._2))).toMap) + def apply[A](kvs: Seq[(Attribute, A)]) = new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap) } class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) @@ -32,10 +31,9 @@ class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2) - override def + [B1 >: A](kv: (Attribute, B1)): Map[Attribute, B1] = - (baseMap.map(_._2) + kv).toMap + override def + [B1 >: A](kv: (Attribute, B1)): Map[Attribute, B1] = baseMap.values.toMap + kv - override def iterator: Iterator[(Attribute, A)] = baseMap.map(_._2).iterator + override def iterator: Iterator[(Attribute, A)] = baseMap.valuesIterator - override def -(key: Attribute): Map[Attribute, A] = (baseMap.map(_._2) - key).toMap + override def -(key: Attribute): Map[Attribute, A] = baseMap.values.toMap - key } 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 882e9c6110089..ed578e081be73 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 @@ -26,25 +26,24 @@ import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.catalyst.trees +/** + * Estimates of various statistics. The default estimation logic simply lazily multiplies the + * corresponding statistic produced by the children. To override this behavior, override + * `statistics` and assign it an overriden version of `Statistics`. + * + * '''NOTE''': concrete and/or overriden versions of statistics fields should pay attention to the + * performance of the implementations. The reason is that estimations might get triggered in + * performance-critical processes, such as query plan planning. + * + * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it + * defaults to the product of children's `sizeInBytes`. + */ +private[sql] case class Statistics(sizeInBytes: BigInt) + abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { self: Product => - /** - * Estimates of various statistics. The default estimation logic simply lazily multiplies the - * corresponding statistic produced by the children. To override this behavior, override - * `statistics` and assign it an overriden version of `Statistics`. - * - * '''NOTE''': concrete and/or overriden versions of statistics fields should pay attention to the - * performance of the implementations. The reason is that estimations might get triggered in - * performance-critical processes, such as query plan planning. - * - * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it - * defaults to the product of children's `sizeInBytes`. - */ - case class Statistics( - sizeInBytes: BigInt - ) - lazy val statistics: Statistics = { + def statistics: Statistics = { if (children.size == 0) { throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index b34ab255d084a..b9f9f8270045c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -24,11 +24,13 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, Attri import org.apache.spark.sql.catalyst.types._ private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { - val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = false)() - val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = false)() - val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() + val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() + val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = true)() + val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() + val count = AttributeReference(a.name + ".count", IntegerType, nullable = false)() + val sizeInBytes = AttributeReference(a.name + ".sizeInBytes", LongType, nullable = false)() - val schema = Seq(lowerBound, upperBound, nullCount) + val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes) } private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { @@ -45,10 +47,21 @@ private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Seri * brings significant performance penalty. */ private[sql] sealed trait ColumnStats extends Serializable { + protected var count = 0 + protected var nullCount = 0 + protected var sizeInBytes = 0L + /** * Gathers statistics information from `row(ordinal)`. */ - def gatherStats(row: Row, ordinal: Int): Unit + def gatherStats(row: Row, ordinal: Int): Unit = { + if (row.isNullAt(ordinal)) { + nullCount += 1 + // 4 bytes for null position + sizeInBytes += 4 + } + count += 1 + } /** * Column statistics represented as a single row, currently including closed lower bound, closed @@ -65,163 +78,154 @@ private[sql] class NoopColumnStats extends ColumnStats { } private[sql] class ByteColumnStats extends ColumnStats { - var upper = Byte.MinValue - var lower = Byte.MaxValue - var nullCount = 0 + protected var upper = Byte.MinValue + protected var lower = Byte.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getByte(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += BYTE.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class ShortColumnStats extends ColumnStats { - var upper = Short.MinValue - var lower = Short.MaxValue - var nullCount = 0 + protected var upper = Short.MinValue + protected var lower = Short.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getShort(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += SHORT.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class LongColumnStats extends ColumnStats { - var upper = Long.MinValue - var lower = Long.MaxValue - var nullCount = 0 + protected var upper = Long.MinValue + protected var lower = Long.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getLong(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += LONG.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class DoubleColumnStats extends ColumnStats { - var upper = Double.MinValue - var lower = Double.MaxValue - var nullCount = 0 + protected var upper = Double.MinValue + protected var lower = Double.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getDouble(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += DOUBLE.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class FloatColumnStats extends ColumnStats { - var upper = Float.MinValue - var lower = Float.MaxValue - var nullCount = 0 + protected var upper = Float.MinValue + protected var lower = Float.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getFloat(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += FLOAT.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class IntColumnStats extends ColumnStats { - var upper = Int.MinValue - var lower = Int.MaxValue - var nullCount = 0 + protected var upper = Int.MinValue + protected var lower = Int.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getInt(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += INT.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class StringColumnStats extends ColumnStats { - var upper: String = null - var lower: String = null - var nullCount = 0 + protected var upper: String = null + protected var lower: String = null override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getString(ordinal) if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value - } else { - nullCount += 1 + sizeInBytes += STRING.actualSize(row, ordinal) } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class DateColumnStats extends ColumnStats { - var upper: Date = null - var lower: Date = null - var nullCount = 0 + protected var upper: Date = null + protected var lower: Date = null override def gatherStats(row: Row, ordinal: Int) { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row(ordinal).asInstanceOf[Date] if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value - } else { - nullCount += 1 + sizeInBytes += DATE.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class TimestampColumnStats extends ColumnStats { - var upper: Timestamp = null - var lower: Timestamp = null - var nullCount = 0 + protected var upper: Timestamp = null + protected var lower: Timestamp = null override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row(ordinal).asInstanceOf[Timestamp] if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value - } else { - nullCount += 1 + sizeInBytes += TIMESTAMP.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 22ab0e2613f21..ee63134f56d8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -19,13 +19,15 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.execution.{LeafNode, SparkPlan} import org.apache.spark.storage.StorageLevel @@ -45,15 +47,51 @@ private[sql] case class InMemoryRelation( useCompression: Boolean, batchSize: Int, storageLevel: StorageLevel, - child: SparkPlan) - (private var _cachedColumnBuffers: RDD[CachedBatch] = null) + child: SparkPlan)( + private var _cachedColumnBuffers: RDD[CachedBatch] = null, + private var _statistics: Statistics = null) extends LogicalPlan with MultiInstanceRelation { - override lazy val statistics = - Statistics(sizeInBytes = child.sqlContext.defaultSizeInBytes) + private val batchStats = + child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) val partitionStatistics = new PartitionStatistics(output) + private def computeSizeInBytes = { + val sizeOfRow: Expression = + BindReferences.bindReference( + output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), + partitionStatistics.schema) + + batchStats.value.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum + } + + // Statistics propagation contracts: + // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data + // 2. Only propagate statistics when `_statistics` is non-null + private def statisticsToBePropagated = if (_statistics == null) { + val updatedStats = statistics + if (_statistics == null) null else updatedStats + } else { + _statistics + } + + override def statistics = if (_statistics == null) { + if (batchStats.value.isEmpty) { + // Underlying columnar RDD hasn't been materialized, no useful statistics information + // available, return the default statistics. + Statistics(sizeInBytes = child.sqlContext.defaultSizeInBytes) + } else { + // Underlying columnar RDD has been materialized, required information has also been collected + // via the `batchStats` accumulator, compute the final statistics, and update `_statistics`. + _statistics = Statistics(sizeInBytes = computeSizeInBytes) + _statistics + } + } else { + // Pre-computed statistics + _statistics + } + // If the cached column buffers were not passed in, we calculate them in the constructor. // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { @@ -91,6 +129,7 @@ private[sql] case class InMemoryRelation( val stats = Row.fromSeq( columnBuilders.map(_.columnStats.collectedStatistics).foldLeft(Seq.empty[Any])(_ ++ _)) + batchStats += stats CachedBatch(columnBuilders.map(_.build().array()), stats) } @@ -104,7 +143,8 @@ private[sql] case class InMemoryRelation( def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( - newOutput, useCompression, batchSize, storageLevel, child)(_cachedColumnBuffers) + newOutput, useCompression, batchSize, storageLevel, child)( + _cachedColumnBuffers, statisticsToBePropagated) } override def children = Seq.empty @@ -116,7 +156,8 @@ private[sql] case class InMemoryRelation( batchSize, storageLevel, child)( - _cachedColumnBuffers).asInstanceOf[this.type] + _cachedColumnBuffers, + statisticsToBePropagated).asInstanceOf[this.type] } def cachedColumnBuffers = _cachedColumnBuffers @@ -132,6 +173,8 @@ private[sql] case class InMemoryColumnarTableScan( override def output: Seq[Attribute] = attributes + private def statsFor(a: Attribute) = relation.partitionStatistics.forAttribute(a) + // Returned filter predicate should return false iff it is impossible for the input expression // to evaluate to `true' based on statistics collected about this partition batch. val buildFilter: PartialFunction[Expression, Expression] = { @@ -144,44 +187,24 @@ private[sql] case class InMemoryColumnarTableScan( buildFilter(lhs) || buildFilter(rhs) case EqualTo(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound <= l && l <= aStats.upperBound - + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound case EqualTo(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound <= l && l <= aStats.upperBound - - case LessThan(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound < l - - case LessThan(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - l < aStats.upperBound - - case LessThanOrEqual(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound <= l + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case LessThanOrEqual(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - l <= aStats.upperBound + case LessThan(a: AttributeReference, l: Literal) => statsFor(a).lowerBound < l + case LessThan(l: Literal, a: AttributeReference) => l < statsFor(a).upperBound - case GreaterThan(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - l < aStats.upperBound + case LessThanOrEqual(a: AttributeReference, l: Literal) => statsFor(a).lowerBound <= l + case LessThanOrEqual(l: Literal, a: AttributeReference) => l <= statsFor(a).upperBound - case GreaterThan(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound < l + case GreaterThan(a: AttributeReference, l: Literal) => l < statsFor(a).upperBound + case GreaterThan(l: Literal, a: AttributeReference) => statsFor(a).lowerBound < l - case GreaterThanOrEqual(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - l <= aStats.upperBound + case GreaterThanOrEqual(a: AttributeReference, l: Literal) => l <= statsFor(a).upperBound + case GreaterThanOrEqual(l: Literal, a: AttributeReference) => statsFor(a).lowerBound <= l - case GreaterThanOrEqual(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound <= l + case IsNull(a: Attribute) => statsFor(a).nullCount > 0 + case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 } val partitionFilters = { 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 2ddf513b6fc98..04c51a1ee4b97 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 @@ -17,16 +17,13 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -import scala.reflect.runtime.universe.TypeTag - import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} +import org.apache.spark.sql.{Row, SQLContext} /** * :: DeveloperApi :: @@ -100,7 +97,7 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQ override final def newInstance(): this.type = { SparkLogicalPlan( alreadyPlanned match { - case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) + case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance()), rdd) case _ => sys.error("Multiple instance of the same relation detected.") })(sqlContext).asInstanceOf[this.type] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 5ae768293a22e..82130b5459174 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -22,7 +22,6 @@ import java.io.IOException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.permission.FsAction - import parquet.hadoop.ParquetOutputFormat import parquet.hadoop.metadata.CompressionCodecName import parquet.schema.MessageType @@ -30,7 +29,7 @@ import parquet.schema.MessageType import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} /** * Relation that consists of data stored in a Parquet columnar format. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index da5a358df3b1d..1a5d87d5240e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} +import org.apache.spark.sql.columnar._ import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.storage.{StorageLevel, RDDBlockId} @@ -234,4 +234,13 @@ class CachedTableSuite extends QueryTest { uncacheTable("testData") assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } + + test("InMemoryRelation statistics") { + sql("CACHE TABLE testData") + table("testData").queryExecution.withCachedData.collect { + case cached: InMemoryRelation => + val actualSizeInBytes = (1 to 100).map(i => INT.defaultSize + i.toString.length + 4).sum + assert(cached.statistics.sizeInBytes === actualSizeInBytes) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 10b7979df7375..1c21afc17e25e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -28,40 +28,40 @@ import org.apache.spark.sql.test.TestSQLContext._ case class TestData(key: Int, value: String) object TestData { - val testData: SchemaRDD = TestSQLContext.sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))) + val testData = TestSQLContext.sparkContext.parallelize( + (1 to 100).map(i => TestData(i, i.toString))).toSchemaRDD testData.registerTempTable("testData") case class LargeAndSmallInts(a: Int, b: Int) - val largeAndSmallInts: SchemaRDD = + val largeAndSmallInts = TestSQLContext.sparkContext.parallelize( LargeAndSmallInts(2147483644, 1) :: LargeAndSmallInts(1, 2) :: LargeAndSmallInts(2147483645, 1) :: LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: - LargeAndSmallInts(3, 2) :: Nil) + LargeAndSmallInts(3, 2) :: Nil).toSchemaRDD largeAndSmallInts.registerTempTable("largeAndSmallInts") case class TestData2(a: Int, b: Int) - val testData2: SchemaRDD = + val testData2 = TestSQLContext.sparkContext.parallelize( TestData2(1, 1) :: TestData2(1, 2) :: TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: - TestData2(3, 2) :: Nil) + TestData2(3, 2) :: Nil).toSchemaRDD testData2.registerTempTable("testData2") case class BinaryData(a: Array[Byte], b: Int) - val binaryData: SchemaRDD = + val binaryData = TestSQLContext.sparkContext.parallelize( BinaryData("12".getBytes(), 1) :: BinaryData("22".getBytes(), 5) :: BinaryData("122".getBytes(), 3) :: BinaryData("121".getBytes(), 2) :: - BinaryData("123".getBytes(), 4) :: Nil) + BinaryData("123".getBytes(), 4) :: Nil).toSchemaRDD binaryData.registerTempTable("binaryData") // TODO: There is no way to express null primitives as case classes currently... diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 6bdf741134e2f..a9f0851f8826c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -61,6 +61,12 @@ class ColumnStatsSuite extends FunSuite { assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) assertResult(values.max(ordering), "Wrong upper bound")(stats(1)) assertResult(10, "Wrong null count")(stats(2)) + assertResult(20, "Wrong row count")(stats(3)) + assertResult(stats(4), "Wrong size in bytes") { + rows.map { row => + if (row.isNullAt(0)) 4 else columnType.actualSize(row, 0) + }.sum + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index f53acc8c9f718..9ba3c210171bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -22,8 +22,6 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.sql._ import org.apache.spark.sql.test.TestSQLContext._ -case class IntegerData(i: Int) - class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter { val originalColumnBatchSize = columnBatchSize val originalInMemoryPartitionPruning = inMemoryPartitionPruning @@ -31,8 +29,12 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be override protected def beforeAll(): Unit = { // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch setConf(SQLConf.COLUMN_BATCH_SIZE, "10") - val rawData = sparkContext.makeRDD(1 to 100, 5).map(IntegerData) - rawData.registerTempTable("intData") + + val pruningData = sparkContext.makeRDD((1 to 100).map { key => + val string = if (((key - 1) / 10) % 2 == 0) null else key.toString + TestData(key, string) + }, 5) + pruningData.registerTempTable("pruningData") // Enable in-memory partition pruning setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") @@ -44,48 +46,64 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be } before { - cacheTable("intData") + cacheTable("pruningData") } after { - uncacheTable("intData") + uncacheTable("pruningData") } // Comparisons - checkBatchPruning("i = 1", Seq(1), 1, 1) - checkBatchPruning("1 = i", Seq(1), 1, 1) - checkBatchPruning("i < 12", 1 to 11, 1, 2) - checkBatchPruning("i <= 11", 1 to 11, 1, 2) - checkBatchPruning("i > 88", 89 to 100, 1, 2) - checkBatchPruning("i >= 89", 89 to 100, 1, 2) - checkBatchPruning("12 > i", 1 to 11, 1, 2) - checkBatchPruning("11 >= i", 1 to 11, 1, 2) - checkBatchPruning("88 < i", 89 to 100, 1, 2) - checkBatchPruning("89 <= i", 89 to 100, 1, 2) + checkBatchPruning("SELECT key FROM pruningData WHERE key = 1", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE 1 = key", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 12", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE key <= 11", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE key > 88", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE key >= 89", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE 12 > key", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE 11 >= key", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE 88 < key", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE 89 <= key", 1, 2)(89 to 100) + + // IS NULL + checkBatchPruning("SELECT key FROM pruningData WHERE value IS NULL", 5, 5) { + (1 to 10) ++ (21 to 30) ++ (41 to 50) ++ (61 to 70) ++ (81 to 90) + } + + // IS NOT NULL + checkBatchPruning("SELECT key FROM pruningData WHERE value IS NOT NULL", 5, 5) { + (11 to 20) ++ (31 to 40) ++ (51 to 60) ++ (71 to 80) ++ (91 to 100) + } // Conjunction and disjunction - checkBatchPruning("i > 8 AND i <= 21", 9 to 21, 2, 3) - checkBatchPruning("i < 2 OR i > 99", Seq(1, 100), 2, 2) - checkBatchPruning("i < 2 OR (i > 78 AND i < 92)", Seq(1) ++ (79 to 91), 3, 4) - checkBatchPruning("NOT (i < 88)", 88 to 100, 1, 2) + checkBatchPruning("SELECT key FROM pruningData WHERE key > 8 AND key <= 21", 2, 3)(9 to 21) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR key > 99", 2, 2)(Seq(1, 100)) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR (key > 78 AND key < 92)", 3, 4) { + Seq(1) ++ (79 to 91) + } // With unsupported predicate - checkBatchPruning("i < 12 AND i IS NOT NULL", 1 to 11, 1, 2) - checkBatchPruning(s"NOT (i in (${(1 to 30).mkString(",")}))", 31 to 100, 5, 10) + checkBatchPruning("SELECT key FROM pruningData WHERE NOT (key < 88)", 1, 2)(88 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 12 AND key IS NOT NULL", 1, 2)(1 to 11) + + { + val seq = (1 to 30).mkString(", ") + checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq))", 5, 10)(31 to 100) + } def checkBatchPruning( - filter: String, - expectedQueryResult: Seq[Int], + query: String, expectedReadPartitions: Int, - expectedReadBatches: Int): Unit = { + expectedReadBatches: Int)( + expectedQueryResult: => Seq[Int]): Unit = { - test(filter) { - val query = sql(s"SELECT * FROM intData WHERE $filter") + test(query) { + val schemaRdd = sql(query) assertResult(expectedQueryResult.toArray, "Wrong query result") { - query.collect().map(_.head).toArray + schemaRdd.collect().map(_.head).toArray } - val (readPartitions, readBatches) = query.queryExecution.executedPlan.collect { + val (readPartitions, readBatches) = schemaRdd.queryExecution.executedPlan.collect { case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) }.head diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index f14ffca0e4d35..a5af71acfc79a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -76,4 +76,24 @@ class PlannerSuite extends FunSuite { setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) } + + test("InMemoryRelation statistics propagation") { + val origThreshold = autoBroadcastJoinThreshold + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, 81920.toString) + + testData.limit(3).registerTempTable("tiny") + sql("CACHE TABLE tiny") + + val a = testData.as('a) + val b = table("tiny").as('b) + val planned = a.join(b, Inner, Some("a.key".attr === "b.key".attr)).queryExecution.executedPlan + + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } + val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } + + assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") + assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") + + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) + } } From 89e8a5d8ba57255c46cadfc803fcddeeec93325f Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sun, 26 Oct 2014 16:24:50 -0700 Subject: [PATCH 031/115] [SPARK-3997][Build]scalastyle should output the error location Author: GuoQiang Li Closes #2846 from witgo/SPARK-3997 and squashes the following commits: d6a57f8 [GuoQiang Li] scalastyle should output the error location --- dev/scalastyle | 2 +- project/SparkBuild.scala | 3 +-- project/plugins.sbt | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/dev/scalastyle b/dev/scalastyle index c3b356bcb3c06..ed1b6b730af6e 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -25,7 +25,7 @@ echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn- echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ >> scalastyle.txt -ERRORS=$(cat scalastyle.txt | grep -e "\") +ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') rm scalastyle.txt if test ! -z "$ERRORS"; then diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ea04473854007..6d5eb681c6131 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -23,7 +23,6 @@ import sbt.Classpaths.publishTask import sbt.Keys._ import sbtunidoc.Plugin.genjavadocSettings import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion -import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} import net.virtualvoid.sbt.graph.Plugin.graphSettings @@ -111,7 +110,7 @@ object SparkBuild extends PomBuild { lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") - lazy val sharedSettings = graphSettings ++ ScalaStyleSettings ++ genjavadocSettings ++ Seq ( + lazy val sharedSettings = graphSettings ++ genjavadocSettings ++ Seq ( javaHome := Properties.envOrNone("JAVA_HOME").map(file), incOptions := incOptions.value.withNameHashing(true), retrieveManaged := true, diff --git a/project/plugins.sbt b/project/plugins.sbt index 9d50a50b109af..ee45b6a51905e 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,7 +19,7 @@ addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.6.0") addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") -addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.5.0") +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.6.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") From dc51f4d6d836fc2c65b0af8db1497427ad2742ec Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 26 Oct 2014 16:27:29 -0700 Subject: [PATCH 032/115] [SQL][DOC] Wrong package name "scala.math.sql" in sql-programming-guide.md In sql-programming-guide.md, there is a wrong package name "scala.math.sql". Author: Kousuke Saruta Closes #2873 from sarutak/wrong-packagename-fix and squashes the following commits: 4d5ecf4 [Kousuke Saruta] Fixed wrong package name in sql-programming-guide.md --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 368c3d0008b07..d4ade939c3a6e 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1215,7 +1215,7 @@ import org.apache.spark.sql._
- + From d518bc24af54a3853fac457e03c70970354150bb Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 26 Oct 2014 16:28:33 -0700 Subject: [PATCH 033/115] [SPARK-3953][SQL][Minor] Confusable variable name. In SqlParser.scala, there is following code. case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => val base = r.getOrElse(NoRelation) val withFilter = f.map(f => Filter(f, base)).getOrElse(base) In the code above, there are 2 variables which have same name "f" in near place. One is receiver "f" and other is bound variable "f". Author: Kousuke Saruta Closes #2807 from sarutak/SPARK-3953 and squashes the following commits: 4957c32 [Kousuke Saruta] Improved variable name in SqlParser.scala --- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a277684f6327c..9bea990fcee4e 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -142,7 +142,7 @@ class SqlParser extends AbstractSparkSQLParser { (LIMIT ~> expression).? ^^ { case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => val base = r.getOrElse(NoRelation) - val withFilter = f.map(f => Filter(f, base)).getOrElse(base) + val withFilter = f.map(Filter(_, base)).getOrElse(base) val withProjection = g .map(Aggregate(_, assignAliases(p), withFilter)) .getOrElse(Project(assignAliases(p), withFilter)) From 05308426f0f51273be95fb1ca2cb1ec19d83cec8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 26 Oct 2014 16:30:15 -0700 Subject: [PATCH 034/115] [SPARK-4052][SQL] Use scala.collection.Map for pattern matching instead of using Predef.Map (it is scala.collection.immutable.Map) Please check https://issues.apache.org/jira/browse/SPARK-4052 for cases triggering this bug. Author: Yin Huai Closes #2899 from yhuai/SPARK-4052 and squashes the following commits: 1188f70 [Yin Huai] Address liancheng's comments. b6712be [Yin Huai] Use scala.collection.Map instead of Predef.Map (scala.collection.immutable.Map). --- .../spark/sql/catalyst/ScalaReflection.scala | 3 +++ .../scala/org/apache/spark/sql/TestData.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 4 ++++ .../sql/hive/InsertIntoHiveTableSuite.scala | 18 ++++++++++++++++++ 4 files changed, 26 insertions(+), 1 deletion(-) 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 3d4296f9d7068..7d930fccd52d1 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 @@ -27,6 +27,9 @@ import org.apache.spark.sql.catalyst.types._ * Provides experimental support for generating catalyst schemas for scala objects. */ object ScalaReflection { + // 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 import scala.reflect.runtime.universe._ case class Schema(dataType: DataType, nullable: Boolean) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 1c21afc17e25e..6c38575b13a2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -99,7 +99,7 @@ object TestData { ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) arrayData.registerTempTable("arrayData") - case class MapData(data: Map[Int, String]) + case class MapData(data: scala.collection.Map[Int, String]) val mapData = TestSQLContext.sparkContext.parallelize( MapData(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: 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 7db5fd804d6ef..79234f8a66f05 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 @@ -93,6 +93,10 @@ case class InsertIntoHiveTable( (o: Any) => seqAsJavaList(o.asInstanceOf[Seq[_]].map(wrapper)) case moi: MapObjectInspector => + // 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 + val keyWrapper = wrapperFor(moi.getMapKeyObjectInspector) val valueWrapper = wrapperFor(moi.getMapValueObjectInspector) (o: Any) => mapAsJavaMap(o.asInstanceOf[Map[_, _]].map { case (key, value) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 7e323146f9da2..18dc937dd2b27 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.QueryTest +import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.TestHive /* Implicits */ @@ -73,4 +74,21 @@ class InsertIntoHiveTableSuite extends QueryTest { createTable[TestData]("createAndInsertTest") createTable[TestData]("createAndInsertTest") } + + test("SPARK-4052: scala.collection.Map as value type of MapType") { + val schema = StructType(StructField("m", MapType(StringType, StringType), true) :: Nil) + val rowRDD = TestHive.sparkContext.parallelize( + (1 to 100).map(i => Row(scala.collection.mutable.HashMap(s"key$i" -> s"value$i")))) + val schemaRDD = applySchema(rowRDD, schema) + schemaRDD.registerTempTable("tableWithMapValue") + sql("CREATE TABLE hiveTableWithMapValue(m MAP )") + sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") + + checkAnswer( + sql("SELECT * FROM hiveTableWithMapValue"), + rowRDD.collect().toSeq + ) + + sql("DROP TABLE hiveTableWithMapValue") + } } From 0481aaa8d751d0df81a1f79fe3f47ca62afd00e8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 26 Oct 2014 16:32:02 -0700 Subject: [PATCH 035/115] [SPARK-4068][SQL] NPE in jsonRDD schema inference Please refer to added tests for cases that can trigger the bug. JIRA: https://issues.apache.org/jira/browse/SPARK-4068 Author: Yin Huai Closes #2918 from yhuai/SPARK-4068 and squashes the following commits: d360eae [Yin Huai] Handle nulls when building key paths from elements of an array. --- .../org/apache/spark/sql/json/JsonRDD.scala | 4 +- .../org/apache/spark/sql/json/JsonSuite.scala | 35 ++++++++++++++++- .../apache/spark/sql/json/TestJsonData.scala | 39 +++++++++++-------- 3 files changed, 58 insertions(+), 20 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 61ee960aad9d2..bf32da1b7181e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -242,14 +242,14 @@ private[sql] object JsonRDD extends Logging { def buildKeyPathForInnerStructs(v: Any, t: DataType): Seq[(String, DataType)] = t match { case ArrayType(StructType(Nil), containsNull) => { // The elements of this arrays are structs. - v.asInstanceOf[Seq[Map[String, Any]]].flatMap { + v.asInstanceOf[Seq[Map[String, Any]]].flatMap(Option(_)).flatMap { element => allKeysWithValueTypes(element) }.map { case (k, t) => (s"$key.$k", t) } } case ArrayType(t1, containsNull) => - v.asInstanceOf[Seq[Any]].flatMap { + v.asInstanceOf[Seq[Any]].flatMap(Option(_)).flatMap { element => buildKeyPathForInnerStructs(element, t1) } case other => Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 7bb08f1b513ce..41927e83698a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -208,7 +208,7 @@ class JsonSuite extends QueryTest { } test("Complex field and type inferring") { - val jsonSchemaRDD = jsonRDD(complexFieldAndType) + val jsonSchemaRDD = jsonRDD(complexFieldAndType1) val expectedSchema = StructType( StructField("arrayOfArray1", ArrayType(ArrayType(StringType, false), false), true) :: @@ -305,7 +305,7 @@ class JsonSuite extends QueryTest { } ignore("Complex field and type inferring (Ignored)") { - val jsonSchemaRDD = jsonRDD(complexFieldAndType) + val jsonSchemaRDD = jsonRDD(complexFieldAndType1) jsonSchemaRDD.registerTempTable("jsonTable") // Right now, "field1" and "field2" are treated as aliases. We should fix it. @@ -707,4 +707,35 @@ class JsonSuite extends QueryTest { TestSQLContext.setConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD, oldColumnNameOfCorruptRecord) } + + test("SPARK-4068: nulls in arrays") { + val jsonSchemaRDD = jsonRDD(nullsInArrays) + jsonSchemaRDD.registerTempTable("jsonTable") + + val schema = StructType( + StructField("field1", + ArrayType(ArrayType(ArrayType(ArrayType(StringType, false), false), true), false), true) :: + StructField("field2", + ArrayType(ArrayType( + StructType(StructField("Test", IntegerType, true) :: Nil), false), true), true) :: + StructField("field3", + ArrayType(ArrayType( + StructType(StructField("Test", StringType, true) :: Nil), true), false), true) :: + StructField("field4", + ArrayType(ArrayType(ArrayType(IntegerType, false), true), false), true) :: Nil) + + assert(schema === jsonSchemaRDD.schema) + + checkAnswer( + sql( + """ + |SELECT field1, field2, field3, field4 + |FROM jsonTable + """.stripMargin), + Seq(Seq(Seq(null), Seq(Seq(Seq("Test")))), null, null, null) :: + Seq(null, Seq(null, Seq(Seq(1))), null, null) :: + Seq(null, null, Seq(Seq(null), Seq(Seq("2"))), null) :: + Seq(null, null, null, Seq(Seq(null, Seq(1, 2, 3)))) :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index eaca9f0508a12..c204162dd2fc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -32,22 +32,6 @@ object TestJsonData { "null":null }""" :: Nil) - val complexFieldAndType = - TestSQLContext.sparkContext.parallelize( - """{"struct":{"field1": true, "field2": 92233720368547758070}, - "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, - "arrayOfString":["str1", "str2"], - "arrayOfInteger":[1, 2147483647, -2147483648], - "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], - "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], - "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], - "arrayOfBoolean":[true, false, true], - "arrayOfNull":[null, null, null, null], - "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], - "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], - "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] - }""" :: Nil) - val primitiveFieldValueTypeConflict = TestSQLContext.sparkContext.parallelize( """{"num_num_1":11, "num_num_2":null, "num_num_3": 1.1, @@ -83,6 +67,22 @@ object TestJsonData { """{"d":{"field":true}}""" :: """{"e":"str"}""" :: Nil) + val complexFieldAndType1 = + TestSQLContext.sparkContext.parallelize( + """{"struct":{"field1": true, "field2": 92233720368547758070}, + "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, + "arrayOfString":["str1", "str2"], + "arrayOfInteger":[1, 2147483647, -2147483648], + "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], + "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], + "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], + "arrayOfBoolean":[true, false, true], + "arrayOfNull":[null, null, null, null], + "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], + "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], + "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] + }""" :: Nil) + val complexFieldAndType2 = TestSQLContext.sparkContext.parallelize( """{"arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], @@ -137,6 +137,13 @@ object TestJsonData { ]] }""" :: Nil) + val nullsInArrays = + TestSQLContext.sparkContext.parallelize( + """{"field1":[[null], [[["Test"]]]]}""" :: + """{"field2":[null, [{"Test":1}]]}""" :: + """{"field3":[[null], [{"Test":"2"}]]}""" :: + """{"field4":[[null, [1,2,3]]]}""" :: Nil) + val jsonArray = TestSQLContext.sparkContext.parallelize( """[{"a":"str_a_1"}]""" :: From 974d7b238b415791975c60e4e202265d6fd31e2d Mon Sep 17 00:00:00 2001 From: ravipesala Date: Sun, 26 Oct 2014 16:36:11 -0700 Subject: [PATCH 036/115] [SPARK-3483][SQL] Special chars in column names Supporting special chars in column names by using back ticks. Closed https://github.com/apache/spark/pull/2804 and created this PR as it has merge conflicts Author: ravipesala Closes #2927 from ravipesala/SPARK-3483-NEW and squashes the following commits: f6329f3 [ravipesala] Rebased with master --- .../org/apache/spark/sql/catalyst/SparkSQLParser.scala | 2 ++ .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 6 ++++++ 2 files changed, 8 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala index 04467342e6ab5..219322c015bbf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala @@ -75,6 +75,8 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { { case chars => StringLit(chars mkString "") } | '"' ~> chrExcept('"', '\n', EofCh).* <~ '"' ^^ { case chars => StringLit(chars mkString "") } + | '`' ~> chrExcept('`', '\n', EofCh).* <~ '`' ^^ + { case chars => Identifier(chars mkString "") } | EofCh ^^^ EOF | '\'' ~> failure("unclosed string literal") | '"' ~> failure("unclosed string literal") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 3959925a2e529..ffb504b94992f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -748,4 +748,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { """.stripMargin), (1 to 100).map(i => Seq(i, i, i))) } + + test("SPARK-3483 Special chars in column names") { + val data = sparkContext.parallelize(Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) + jsonRDD(data).registerTempTable("records") + sql("SELECT `key?number1` FROM records") + } } From ace41e8bf2f4538115b28b90757b8424ca614682 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 26 Oct 2014 16:40:29 -0700 Subject: [PATCH 037/115] [SPARK-3959][SPARK-3960][SQL] SqlParser fails to parse literal -9223372036854775808 (Long.MinValue). / We can apply unary minus only to literal. SqlParser fails to parse -9223372036854775808 (Long.MinValue) so we cannot write queries such like as follows. SELECT value FROM someTable WHERE value > -9223372036854775808 Additionally, because of the wrong syntax definition, we cannot apply unary minus only to literal. So, we cannot write such expressions. -(value1 + value2) // Parenthesized expressions -column // Columns -MAX(column) // Functions Author: Kousuke Saruta Closes #2816 from sarutak/spark-sql-dsl-improvement2 and squashes the following commits: 32a5005 [Kousuke Saruta] Remove test setting for thriftserver c2bab5e [Kousuke Saruta] Fixed SPARK-3959 and SPARK-3960 --- .../apache/spark/sql/catalyst/SqlParser.scala | 68 +++++++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 129 ++++++++++++++++++ .../org/apache/spark/sql/json/JsonSuite.scala | 13 +- 3 files changed, 189 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 9bea990fcee4e..4e967713ede64 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -303,33 +303,73 @@ class SqlParser extends AbstractSparkSQLParser { CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ { case exp ~ t => Cast(exp, t) } protected lazy val literal: Parser[Literal] = - ( numericLit ^^ { - case i if i.toLong > Int.MaxValue => Literal(i.toLong) - case i => Literal(i.toInt) - } - | NULL ^^^ Literal(null, NullType) - | floatLit ^^ {case f => Literal(f.toDouble) } + ( numericLiteral + | booleanLiteral | stringLit ^^ {case s => Literal(s, StringType) } + | NULL ^^^ Literal(null, NullType) + ) + + protected lazy val booleanLiteral: Parser[Literal] = + ( TRUE ^^^ Literal(true, BooleanType) + | FALSE ^^^ Literal(false, BooleanType) + ) + + protected lazy val numericLiteral: Parser[Literal] = + signedNumericLiteral | unsignedNumericLiteral + + protected lazy val sign: Parser[String] = + "+" | "-" + + protected lazy val signedNumericLiteral: Parser[Literal] = + ( sign ~ numericLit ^^ { case s ~ l => Literal(toNarrowestIntegerType(s + l)) } + | sign ~ floatLit ^^ { case s ~ f => Literal((s + f).toDouble) } + ) + + protected lazy val unsignedNumericLiteral: Parser[Literal] = + ( numericLit ^^ { n => Literal(toNarrowestIntegerType(n)) } + | floatLit ^^ { f => Literal(f.toDouble) } ) + private val longMax = BigDecimal(s"${Long.MaxValue}") + private val longMin = BigDecimal(s"${Long.MinValue}") + private val intMax = BigDecimal(s"${Int.MaxValue}") + private val intMin = BigDecimal(s"${Int.MinValue}") + + private def toNarrowestIntegerType(value: String) = { + val bigIntValue = BigDecimal(value) + + bigIntValue match { + case v if v < longMin || v > longMax => v + case v if v < intMin || v > intMax => v.toLong + case v => v.toInt + } + } + protected lazy val floatLit: Parser[String] = - elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) + ( "." ~> unsignedNumericLiteral ^^ { u => "0." + u } + | elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) + ) + + protected lazy val baseExpression: Parser[Expression] = + ( "*" ^^^ Star(None) + | primary + ) - protected lazy val baseExpression: PackratParser[Expression] = - ( expression ~ ("[" ~> expression <~ "]") ^^ + protected lazy val signedPrimary: Parser[Expression] = + sign ~ primary ^^ { case s ~ e => if (s == "-") UnaryMinus(e) else e} + + protected lazy val primary: PackratParser[Expression] = + ( literal + | expression ~ ("[" ~> expression <~ "]") ^^ { case base ~ ordinal => GetItem(base, ordinal) } | (expression <~ ".") ~ ident ^^ { case base ~ fieldName => GetField(base, fieldName) } - | TRUE ^^^ Literal(true, BooleanType) - | FALSE ^^^ Literal(false, BooleanType) | cast | "(" ~> expression <~ ")" | function - | "-" ~> literal ^^ UnaryMinus | dotExpressionHeader | ident ^^ UnresolvedAttribute - | "*" ^^^ Star(None) - | literal + | signedPrimary ) protected lazy val dotExpressionHeader: Parser[Expression] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index ffb504b94992f..12e1cfc1cb7eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -738,6 +738,135 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) } + test("Test to check we can use Long.MinValue") { + checkAnswer( + sql(s"SELECT ${Long.MinValue} FROM testData ORDER BY key LIMIT 1"), Long.MinValue + ) + + checkAnswer( + sql(s"SELECT key FROM testData WHERE key > ${Long.MinValue}"), (1 to 100).map(Row(_)).toSeq + ) + } + + test("Floating point number format") { + checkAnswer( + sql("SELECT 0.3"), 0.3 + ) + + checkAnswer( + sql("SELECT -0.8"), -0.8 + ) + + checkAnswer( + sql("SELECT .5"), 0.5 + ) + + checkAnswer( + sql("SELECT -.18"), -0.18 + ) + } + + test("Auto cast integer type") { + checkAnswer( + sql(s"SELECT ${Int.MaxValue + 1L}"), Int.MaxValue + 1L + ) + + checkAnswer( + sql(s"SELECT ${Int.MinValue - 1L}"), Int.MinValue - 1L + ) + + checkAnswer( + sql("SELECT 9223372036854775808"), BigDecimal("9223372036854775808") + ) + + checkAnswer( + sql("SELECT -9223372036854775809"), BigDecimal("-9223372036854775809") + ) + } + + test("Test to check we can apply sign to expression") { + + checkAnswer( + sql("SELECT -100"), -100 + ) + + checkAnswer( + sql("SELECT +230"), 230 + ) + + checkAnswer( + sql("SELECT -5.2"), -5.2 + ) + + checkAnswer( + sql("SELECT +6.8"), 6.8 + ) + + checkAnswer( + sql("SELECT -key FROM testData WHERE key = 2"), -2 + ) + + checkAnswer( + sql("SELECT +key FROM testData WHERE key = 3"), 3 + ) + + checkAnswer( + sql("SELECT -(key + 1) FROM testData WHERE key = 1"), -2 + ) + + checkAnswer( + sql("SELECT - key + 1 FROM testData WHERE key = 10"), -9 + ) + + checkAnswer( + sql("SELECT +(key + 5) FROM testData WHERE key = 5"), 10 + ) + + checkAnswer( + sql("SELECT -MAX(key) FROM testData"), -100 + ) + + checkAnswer( + sql("SELECT +MAX(key) FROM testData"), 100 + ) + + checkAnswer( + sql("SELECT - (-10)"), 10 + ) + + checkAnswer( + sql("SELECT + (-key) FROM testData WHERE key = 32"), -32 + ) + + checkAnswer( + sql("SELECT - (+Max(key)) FROM testData"), -100 + ) + + checkAnswer( + sql("SELECT - - 3"), 3 + ) + + checkAnswer( + sql("SELECT - + 20"), -20 + ) + + checkAnswer( + sql("SELEcT - + 45"), -45 + ) + + checkAnswer( + sql("SELECT + + 100"), 100 + ) + + checkAnswer( + sql("SELECT - - Max(key) FROM testData"), 100 + ) + + checkAnswer( + sql("SELECT + - key FROM testData WHERE key = 33"), -33 + ) + } + test("Multiple join") { checkAnswer( sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 41927e83698a5..1ae75546aada1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -380,6 +380,12 @@ class JsonSuite extends QueryTest { 92233720368547758071.2 ) + // Number and String conflict: resolve the type as number in this query. + checkAnswer( + sql("select num_str + 1.2 from jsonTable where num_str > 92233720368547758060"), + BigDecimal("92233720368547758061.2").toDouble + ) + // String and Boolean conflict: resolve the type as string. checkAnswer( sql("select * from jsonTable where str_bool = 'str1'"), @@ -415,13 +421,6 @@ class JsonSuite extends QueryTest { false ) - // Right now, we have a parsing error. - // Number and String conflict: resolve the type as number in this query. - checkAnswer( - sql("select num_str + 1.2 from jsonTable where num_str > 92233720368547758060"), - BigDecimal("92233720368547758061.2") - ) - // The plan of the following DSL is // Project [(CAST(num_str#65:4, DoubleType) + 1.2) AS num#78] // Filter (CAST(CAST(num_str#65:4, DoubleType), DecimalType) > 92233720368547758060) From 3a9d66cf59ab7c9aee090e4c6067c73510e2ac26 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 26 Oct 2014 16:54:07 -0700 Subject: [PATCH 038/115] [SPARK-4061][SQL] We cannot use EOL character in the operand of LIKE predicate. We cannot use EOL character like \n or \r in the operand of LIKE predicate. So following condition is never true. -- someStr is 'hoge\nfuga' where someStr LIKE 'hoge_fuga' Author: Kousuke Saruta Closes #2908 from sarutak/spark-sql-like-match-modification and squashes the following commits: d15798b [Kousuke Saruta] Remove test setting for thriftserver f99a2f4 [Kousuke Saruta] Fixed LIKE predicate so that we can use EOL character as in a operand --- .../expressions/stringOperations.scala | 42 +++++++++---------- .../ExpressionEvaluationSuite.scala | 6 +++ 2 files changed, 25 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index c2a3a5ca3ca8b..f6349767764a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -102,31 +102,27 @@ case class Like(left: Expression, right: Expression) // replace the _ with .{1} exactly match 1 time of any character // replace the % with .*, match 0 or more times with any character - override def escape(v: String) = { - val sb = new StringBuilder() - var i = 0; - while (i < v.length) { - // Make a special case for "\\_" and "\\%" - val n = v.charAt(i); - if (n == '\\' && i + 1 < v.length && (v.charAt(i + 1) == '_' || v.charAt(i + 1) == '%')) { - sb.append(v.charAt(i + 1)) - i += 1 - } else { - if (n == '_') { - sb.append("."); - } else if (n == '%') { - sb.append(".*"); - } else { - sb.append(Pattern.quote(Character.toString(n))); - } - } - - i += 1 + override def escape(v: String) = + if (!v.isEmpty) { + "(?s)" + (' ' +: v.init).zip(v).flatMap { + case (prev, '\\') => "" + case ('\\', c) => + c match { + case '_' => "_" + case '%' => "%" + case _ => Pattern.quote("\\" + c) + } + case (prev, c) => + c match { + case '_' => "." + case '%' => ".*" + case _ => Pattern.quote(Character.toString(c)) + } + }.mkString + } else { + v } - sb.toString() - } - override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index f134d73450515..53c53481f984e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -191,6 +191,9 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like "a%", true) checkEvaluation("abc" like "b%", false) checkEvaluation("abc" like "bc%", false) + checkEvaluation("a\nb" like "a_b", true) + checkEvaluation("ab" like "a%b", true) + checkEvaluation("a\nb" like "a%b", true) } test("LIKE Non-literal Regular Expression") { @@ -207,6 +210,9 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) + checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a_b"))) + checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) + checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) } From f4e8c289d899fc03ee3cebd01d810f207868b448 Mon Sep 17 00:00:00 2001 From: scwf Date: Sun, 26 Oct 2014 16:56:03 -0700 Subject: [PATCH 039/115] [SPARK-4042][SQL] Append columns ids and names before broadcast Append columns ids and names before broadcast ```hiveExtraConf``` in ```HadoopTableReader```. Author: scwf Closes #2885 from scwf/HadoopTableReader and squashes the following commits: a8c498c [scwf] append columns ids and names before broadcast --- .../org/apache/spark/sql/hive/execution/HiveTableScan.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 85965a6ea095a..b7f3ade4ea819 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 @@ -68,6 +68,9 @@ case class HiveTableScan( @transient private[this] val hiveExtraConf = new HiveConf(context.hiveconf) + // append columns ids and names before broadcast + addColumnMetadataToConf(hiveExtraConf) + @transient private[this] val hadoopReader = new HadoopTableReader(attributes, relation, context, hiveExtraConf) @@ -105,8 +108,6 @@ case class HiveTableScan( hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(",")) } - addColumnMetadataToConf(hiveExtraConf) - /** * Prunes partitions not involve the query plan. * From 6377adaf3212b4facb4af644b70b7e99455cef48 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 26 Oct 2014 18:02:06 -0700 Subject: [PATCH 040/115] [SPARK-3970] Remove duplicate removal of local dirs The shutdown hook of `DiskBlockManager` would remove localDirs. So do not need to register them with `Utils.registerShutdownDeleteDir`. It causes duplicate removal of these local dirs and corresponding exceptions. Author: Liang-Chi Hsieh Closes #2826 from viirya/fix_duplicate_localdir_remove and squashes the following commits: 051d4b5 [Liang-Chi Hsieh] check dir existing and return empty List as default. 2b91a9c [Liang-Chi Hsieh] remove duplicate removal of local dirs. --- .../org/apache/spark/storage/DiskBlockManager.scala | 1 - .../src/main/scala/org/apache/spark/util/Utils.scala | 12 ++++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 6633a1db57e59..99e925328a4b9 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -149,7 +149,6 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } private def addShutdownHook() { - localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") 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 e1dc49238733c..d722ee5a97e94 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -739,11 +739,15 @@ private[spark] object Utils extends Logging { } private def listFilesSafely(file: File): Seq[File] = { - val files = file.listFiles() - if (files == null) { - throw new IOException("Failed to list files for dir: " + file) + if (file.exists()) { + val files = file.listFiles() + if (files == null) { + throw new IOException("Failed to list files for dir: " + file) + } + files + } else { + List() } - files } /** From 9aa340a23fd7532f5e72c3352df92ce3e857fc80 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 27 Oct 2014 08:45:36 -0700 Subject: [PATCH 041/115] [SPARK-4030] Make destroy public for broadcast variables This change makes the destroy function public for broadcast variables. Motivation for the change is described in https://issues.apache.org/jira/browse/SPARK-4030. This patch also logs where destroy was called from if a broadcast variable is used after destruction. Author: Shivaram Venkataraman Closes #2922 from shivaram/broadcast-destroy and squashes the following commits: a11abab [Shivaram Venkataraman] Fix scala style in Utils.scala bed9c9d [Shivaram Venkataraman] Make destroy blocking by default e80c1ab [Shivaram Venkataraman] Make destroy public for broadcast variables Also log where destroy was called from if a broadcast variable is used after destruction. --- .../apache/spark/broadcast/Broadcast.scala | 22 +++++++++++++++++-- .../scala/org/apache/spark/util/Utils.scala | 3 ++- .../spark/broadcast/BroadcastSuite.scala | 20 ++++++++++++++++- 3 files changed, 41 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 15fd30e65761d..87f5cf944ed85 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -20,6 +20,8 @@ package org.apache.spark.broadcast import java.io.Serializable import org.apache.spark.SparkException +import org.apache.spark.Logging +import org.apache.spark.util.Utils import scala.reflect.ClassTag @@ -52,7 +54,7 @@ import scala.reflect.ClassTag * @param id A unique identifier for the broadcast variable. * @tparam T Type of the data contained in the broadcast variable. */ -abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { +abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Logging { /** * Flag signifying whether the broadcast variable is valid @@ -60,6 +62,8 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { */ @volatile private var _isValid = true + private var _destroySite = "" + /** Get the broadcasted value. */ def value: T = { assertValid() @@ -84,13 +88,26 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { doUnpersist(blocking) } + + /** + * Destroy all data and metadata related to this broadcast variable. Use this with caution; + * once a broadcast variable has been destroyed, it cannot be used again. + * This method blocks until destroy has completed + */ + def destroy() { + destroy(blocking = true) + } + /** * Destroy all data and metadata related to this broadcast variable. Use this with caution; * once a broadcast variable has been destroyed, it cannot be used again. + * @param blocking Whether to block until destroy has completed */ private[spark] def destroy(blocking: Boolean) { assertValid() _isValid = false + _destroySite = Utils.getCallSite().shortForm + logInfo("Destroying %s (from %s)".format(toString, _destroySite)) doDestroy(blocking) } @@ -124,7 +141,8 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { /** Check if this broadcast is valid. If not valid, exception is thrown. */ protected def assertValid() { if (!_isValid) { - throw new SparkException("Attempted to use %s after it has been destroyed!".format(toString)) + throw new SparkException( + "Attempted to use %s after it was destroyed (%s) ".format(toString, _destroySite)) } } 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 d722ee5a97e94..84ed5db8f0a53 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -992,7 +992,8 @@ private[spark] object Utils extends Logging { private def coreExclusionFunction(className: String): Boolean = { // A regular expression to match classes of the "core" Spark API that we want to skip when // finding the call site of a method. - val SPARK_CORE_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r + val SPARK_CORE_CLASS_REGEX = + """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?(\.broadcast)?\.[A-Z]""".r val SCALA_CLASS_REGEX = """^scala""".r val isSparkCoreClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined val isScalaClass = SCALA_CLASS_REGEX.findFirstIn(className).isDefined diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index e096c8c3e9b46..1014fd62d9a75 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.broadcast import scala.util.Random -import org.scalatest.FunSuite +import org.scalatest.{Assertions, FunSuite} import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} import org.apache.spark.io.SnappyCompressionCodec @@ -136,6 +136,12 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { test("Unpersisting TorrentBroadcast on executors and driver in distributed mode") { testUnpersistTorrentBroadcast(distributed = true, removeFromDriver = true) } + + test("Using broadcast after destroy prints callsite") { + sc = new SparkContext("local", "test") + testPackage.runCallSiteTest(sc) + } + /** * Verify the persistence of state associated with an HttpBroadcast in either local mode or * local-cluster mode (when distributed = true). @@ -311,3 +317,15 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { conf } } + +package object testPackage extends Assertions { + + def runCallSiteTest(sc: SparkContext) { + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) + val broadcast = sc.broadcast(rdd) + broadcast.destroy() + val thrown = intercept[SparkException] { broadcast.value } + assert(thrown.getMessage.contains("BroadcastSuite.scala")) + } + +} From c9e05ca27c9c702b510d424e3befc87213f24e0f Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 27 Oct 2014 10:02:48 -0700 Subject: [PATCH 042/115] [SPARK-4032] Deprecate YARN alpha support in Spark 1.2 Author: Prashant Sharma Closes #2878 from ScrapCodes/SPARK-4032/deprecate-yarn-alpha and squashes the following commits: 17e9857 [Prashant Sharma] added deperecated comment to Client and ExecutorRunnable. 3a34b1e [Prashant Sharma] Updated docs... 4608dea [Prashant Sharma] [SPARK-4032] Deprecate YARN alpha support in Spark 1.2 --- docs/building-spark.md | 4 +++- .../org/apache/spark/deploy/yarn/Client.scala | 2 ++ .../spark/deploy/yarn/ExecutorRunnable.scala | 2 +- yarn/pom.xml | 22 +++++++++++++++++++ 4 files changed, 28 insertions(+), 2 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 11fd56c145c01..4cc0b1f2e5116 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -67,11 +67,13 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN - +
DecimalType scala.math.sql.BigDecimal scala.math.BigDecimal DecimalType
YARN versionProfile required
0.23.x to 2.1.xyarn-alpha
0.23.x to 2.1.xyarn-alpha (Deprecated.)
2.2.x and lateryarn
+Note: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445). + Examples: {% highlight bash %} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 9c66c785848a5..73b705ba50051 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -35,6 +35,7 @@ import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. */ +@deprecated("use yarn/stable", "1.2.0") private[spark] class Client( val args: ClientArguments, val hadoopConf: Configuration, @@ -131,6 +132,7 @@ object Client { println("WARNING: This client is deprecated and will be removed in a " + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") } + println("WARNING: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445)") // Set an env variable indicating we are running in YARN mode. // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 229b7a09f456b..7ee4b5c842df1 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.spark.{SecurityManager, SparkConf, Logging} - +@deprecated("use yarn/stable", "1.2.0") class ExecutorRunnable( container: Container, conf: Configuration, diff --git a/yarn/pom.xml b/yarn/pom.xml index 8a7035c85e9f1..2885e6607ec24 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -73,6 +73,28 @@ yarn-alpha + + + + maven-antrun-plugin + + + validate + + run + + + + ******************************************************************************************* + ***WARNING***: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445).* + ******************************************************************************************* + + + + + + + alpha From dea302ddbd26b1f20fb8a3979bd1d8e1717479f8 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 27 Oct 2014 10:04:24 -0700 Subject: [PATCH 043/115] SPARK-2621. Update task InputMetrics incrementally The patch takes advantage an API provided in Hadoop 2.5 that allows getting accurate data on Hadoop FileSystem bytes read. It eliminates the old method, which naively accepts the split size as the input bytes. An impact of this change will be that input metrics go away when using against Hadoop versions earlier thatn 2.5. I can add this back in, but my opinion is that no metrics are better than inaccurate metrics. This is difficult to write a test for because we don't usually build against a version of Hadoop that contains the function we need. I've tested it manually on a pseudo-distributed cluster. Author: Sandy Ryza Closes #2087 from sryza/sandy-spark-2621 and squashes the following commits: 23010b8 [Sandy Ryza] Missing style fixes 74fc9bb [Sandy Ryza] Make getFSBytesReadOnThreadCallback private 1ab662d [Sandy Ryza] Clear things up a bit 984631f [Sandy Ryza] Switch from pull to push model and add test 7ef7b22 [Sandy Ryza] Add missing curly braces 219abc9 [Sandy Ryza] Fall back to split size 90dbc14 [Sandy Ryza] SPARK-2621. Update task InputMetrics incrementally --- .../apache/spark/deploy/SparkHadoopUtil.scala | 30 +++++++++++ .../apache/spark/executor/TaskMetrics.scala | 1 - .../org/apache/spark/rdd/HadoopRDD.scala | 48 +++++++++++++---- .../org/apache/spark/rdd/NewHadoopRDD.scala | 48 +++++++++++++---- .../scala/org/apache/spark/util/Utils.scala | 11 ++++ .../spark/metrics/InputMetricsSuite.scala | 53 +++++++++++++++++++ 6 files changed, 170 insertions(+), 21 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index fe0ad9ebbca12..e28eaad8a5180 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -20,12 +20,15 @@ package org.apache.spark.deploy import java.security.PrivilegedExceptionAction import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{Logging, SparkContext, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils import scala.collection.JavaConversions._ @@ -121,6 +124,33 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } + /** + * Returns a function that can be called to find Hadoop FileSystem bytes read. If + * getFSBytesReadOnThreadCallback is called from thread r at time t, the returned callback will + * return the bytes read on r since t. Reflection is required because thread-level FileSystem + * statistics are only available as of Hadoop 2.5 (see HADOOP-10688). + * Returns None if the required method can't be found. + */ + private[spark] def getFSBytesReadOnThreadCallback(path: Path, conf: Configuration) + : Option[() => Long] = { + val qualifiedPath = path.getFileSystem(conf).makeQualified(path) + val scheme = qualifiedPath.toUri().getScheme() + val stats = FileSystem.getAllStatistics().filter(_.getScheme().equals(scheme)) + try { + val threadStats = stats.map(Utils.invoke(classOf[Statistics], _, "getThreadStatistics")) + val statisticsDataClass = + Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") + val getBytesReadMethod = statisticsDataClass.getDeclaredMethod("getBytesRead") + val f = () => threadStats.map(getBytesReadMethod.invoke(_).asInstanceOf[Long]).sum + val baselineBytesRead = f() + Some(() => f() - baselineBytesRead) + } catch { + case e: NoSuchMethodException => { + logDebug("Couldn't find method for retrieving thread-level FileSystem input data", e) + None + } + } + } } object SparkHadoopUtil { 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 3e49b6235aff3..57bc2b40cec44 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -169,7 +169,6 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { var bytesRead: Long = 0L } - /** * :: DeveloperApi :: * Metrics pertaining to shuffle data read in a given task. diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 775141775e06c..946fb5616d3ec 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -46,7 +46,6 @@ import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.util.{NextIterator, Utils} import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation} - /** * A Spark split class that wraps around a Hadoop InputSplit. */ @@ -224,18 +223,18 @@ class HadoopRDD[K, V]( val key: K = reader.createKey() val value: V = reader.createValue() - // Set the task input metrics. val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - try { - /* bytesRead may not exactly equal the bytes read by a task: split boundaries aren't - * always at record boundaries, so tasks may need to read into other splits to complete - * a record. */ - inputMetrics.bytesRead = split.inputSplit.value.getLength() - } catch { - case e: java.io.IOException => - logWarning("Unable to get input size to set InputMetrics for task", e) + // Find a function that will return the FileSystem bytes read by this thread. + val bytesReadCallback = if (split.inputSplit.value.isInstanceOf[FileSplit]) { + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( + split.inputSplit.value.asInstanceOf[FileSplit].getPath, jobConf) + } else { + None + } + if (bytesReadCallback.isDefined) { + context.taskMetrics.inputMetrics = Some(inputMetrics) } - context.taskMetrics.inputMetrics = Some(inputMetrics) + var recordsSinceMetricsUpdate = 0 override def getNext() = { try { @@ -244,12 +243,36 @@ class HadoopRDD[K, V]( case eof: EOFException => finished = true } + + // Update bytes read metric every few records + if (recordsSinceMetricsUpdate == HadoopRDD.RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES + && bytesReadCallback.isDefined) { + recordsSinceMetricsUpdate = 0 + val bytesReadFn = bytesReadCallback.get + inputMetrics.bytesRead = bytesReadFn() + } else { + recordsSinceMetricsUpdate += 1 + } (key, value) } override def close() { try { reader.close() + if (bytesReadCallback.isDefined) { + val bytesReadFn = bytesReadCallback.get + inputMetrics.bytesRead = bytesReadFn() + } else if (split.inputSplit.value.isInstanceOf[FileSplit]) { + // If we can't get the bytes read from the FS stats, fall back to the split size, + // which may be inaccurate. + try { + inputMetrics.bytesRead = split.inputSplit.value.getLength + context.taskMetrics.inputMetrics = Some(inputMetrics) + } catch { + case e: java.io.IOException => + logWarning("Unable to get input size to set InputMetrics for task", e) + } + } } catch { case e: Exception => { if (!Utils.inShutdown()) { @@ -302,6 +325,9 @@ private[spark] object HadoopRDD extends Logging { */ val CONFIGURATION_INSTANTIATION_LOCK = new Object() + /** Update the input bytes read metric each time this number of records has been read */ + val RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES = 256 + /** * The three methods below are helpers for accessing the local map, a property of the SparkEnv of * the local process. diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 0cccdefc5ee09..324563248793c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,6 +25,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.input.FileSplit import org.apache.spark.annotation.DeveloperApi import org.apache.spark.input.WholeTextFileInputFormat @@ -36,6 +37,7 @@ import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil private[spark] class NewHadoopPartition( rddId: Int, @@ -118,21 +120,22 @@ class NewHadoopRDD[K, V]( reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - try { - /* bytesRead may not exactly equal the bytes read by a task: split boundaries aren't - * always at record boundaries, so tasks may need to read into other splits to complete - * a record. */ - inputMetrics.bytesRead = split.serializableHadoopSplit.value.getLength() - } catch { - case e: Exception => - logWarning("Unable to get input split size in order to set task input bytes", e) + // Find a function that will return the FileSystem bytes read by this thread. + val bytesReadCallback = if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( + split.serializableHadoopSplit.value.asInstanceOf[FileSplit].getPath, conf) + } else { + None + } + if (bytesReadCallback.isDefined) { + context.taskMetrics.inputMetrics = Some(inputMetrics) } - context.taskMetrics.inputMetrics = Some(inputMetrics) // Register an on-task-completion callback to close the input stream. context.addTaskCompletionListener(context => close()) var havePair = false var finished = false + var recordsSinceMetricsUpdate = 0 override def hasNext: Boolean = { if (!finished && !havePair) { @@ -147,12 +150,39 @@ class NewHadoopRDD[K, V]( throw new java.util.NoSuchElementException("End of stream") } havePair = false + + // Update bytes read metric every few records + if (recordsSinceMetricsUpdate == HadoopRDD.RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES + && bytesReadCallback.isDefined) { + recordsSinceMetricsUpdate = 0 + val bytesReadFn = bytesReadCallback.get + inputMetrics.bytesRead = bytesReadFn() + } else { + recordsSinceMetricsUpdate += 1 + } + (reader.getCurrentKey, reader.getCurrentValue) } private def close() { try { reader.close() + + // Update metrics with final amount + if (bytesReadCallback.isDefined) { + val bytesReadFn = bytesReadCallback.get + inputMetrics.bytesRead = bytesReadFn() + } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { + // If we can't get the bytes read from the FS stats, fall back to the split size, + // which may be inaccurate. + try { + inputMetrics.bytesRead = split.serializableHadoopSplit.value.getLength + context.taskMetrics.inputMetrics = Some(inputMetrics) + } catch { + case e: java.io.IOException => + logWarning("Unable to get input size to set InputMetrics for task", e) + } + } } catch { case e: Exception => { if (!Utils.inShutdown()) { 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 84ed5db8f0a53..93ac9f1c33ea8 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1673,6 +1673,17 @@ private[spark] object Utils extends Logging { PropertyConfigurator.configure(pro) } + def invoke( + clazz: Class[_], + obj: AnyRef, + methodName: String, + args: (Class[_], AnyRef)*): AnyRef = { + val (types, values) = args.unzip + val method = clazz.getDeclaredMethod(methodName, types: _*) + method.setAccessible(true) + method.invoke(obj, values.toSeq: _*) + } + } /** diff --git a/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala new file mode 100644 index 0000000000000..33bd1afea2470 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.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.metrics + +import org.scalatest.FunSuite + +import org.apache.spark.SharedSparkContext +import org.apache.spark.scheduler.{SparkListenerTaskEnd, SparkListener} + +import scala.collection.mutable.ArrayBuffer + +import java.io.{FileWriter, PrintWriter, File} + +class InputMetricsSuite extends FunSuite with SharedSparkContext { + test("input metrics when reading text file") { + val file = new File(getClass.getSimpleName + ".txt") + val pw = new PrintWriter(new FileWriter(file)) + pw.println("some stuff") + pw.println("some other stuff") + pw.println("yet more stuff") + pw.println("too much stuff") + pw.close() + file.deleteOnExit() + + val taskBytesRead = new ArrayBuffer[Long]() + sc.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead + } + }) + sc.textFile("file://" + file.getAbsolutePath, 2).count() + + // Wait for task end events to come in + sc.listenerBus.waitUntilEmpty(500) + assert(taskBytesRead.length == 2) + assert(taskBytesRead.sum == file.length()) + } +} From 1d7bcc88401d66c8d17a075355acfc25a8b7615c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 27 Oct 2014 10:06:05 -0700 Subject: [PATCH 044/115] [SQL] Fixes caching related JoinSuite failure PR #2860 refines in-memory table statistics and enables broader broadcasted hash join optimization for in-memory tables. This makes `JoinSuite` fail when some test suite caches test table `testData` and gets executed before `JoinSuite`. Because expected `ShuffledHashJoin`s are optimized to `BroadcastedHashJoin` according to collected in-memory table statistics. This PR fixes this issue by clearing the cache before testing join operator selection. A separate test case is also added to test broadcasted hash join operator selection. Author: Cheng Lian Closes #2960 from liancheng/fix-join-suite and squashes the following commits: 715b2de [Cheng Lian] Fixes caching related JoinSuite failure --- .../org/apache/spark/sql/JoinSuite.scala | 117 ++++++++++-------- .../scala/org/apache/spark/sql/TestData.scala | 4 +- 2 files changed, 64 insertions(+), 57 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 07f4d2946c1b5..8b4cf5bac0187 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -19,17 +19,13 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner, LeftSemi} -import org.apache.spark.sql.execution._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, RightOuter} import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ class JoinSuite extends QueryTest with BeforeAndAfterEach { - // Ensures tables are loaded. TestData @@ -41,54 +37,65 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { assert(planned.size === 1) } - test("join operator selection") { - def assertJoin(sqlString: String, c: Class[_]): Any = { - val rdd = sql(sqlString) - val physical = rdd.queryExecution.sparkPlan - val operators = physical.collect { - case j: ShuffledHashJoin => j - case j: HashOuterJoin => j - case j: LeftSemiJoinHash => j - case j: BroadcastHashJoin => j - case j: LeftSemiJoinBNL => j - case j: CartesianProduct => j - case j: BroadcastNestedLoopJoin => j - } - - assert(operators.size === 1) - if (operators(0).getClass() != c) { - fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") - } + def assertJoin(sqlString: String, c: Class[_]): Any = { + val rdd = sql(sqlString) + val physical = rdd.queryExecution.sparkPlan + val operators = physical.collect { + case j: ShuffledHashJoin => j + case j: HashOuterJoin => j + case j: LeftSemiJoinHash => j + case j: BroadcastHashJoin => j + case j: LeftSemiJoinBNL => j + case j: CartesianProduct => j + case j: BroadcastNestedLoopJoin => j + } + + assert(operators.size === 1) + if (operators(0).getClass() != c) { + fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") } + } - val cases1 = Seq( - ("SELECT * FROM testData left semi join testData2 ON key = a", classOf[LeftSemiJoinHash]), - ("SELECT * FROM testData left semi join testData2", classOf[LeftSemiJoinBNL]), - ("SELECT * FROM testData join testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData join testData2 where key=2", classOf[CartesianProduct]), - ("SELECT * FROM testData left join testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData right join testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData full outer join testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData left join testData2 where key=2", classOf[CartesianProduct]), - ("SELECT * FROM testData right join testData2 where key=2", classOf[CartesianProduct]), - ("SELECT * FROM testData full outer join testData2 where key=2", classOf[CartesianProduct]), - ("SELECT * FROM testData join testData2 where key>a", classOf[CartesianProduct]), - ("SELECT * FROM testData full outer join testData2 where key>a", classOf[CartesianProduct]), - ("SELECT * FROM testData join testData2 ON key = a", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a and key=2", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key=2", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData left join testData2 ON key = a", classOf[HashOuterJoin]), - ("SELECT * FROM testData right join testData2 ON key = a where key=2", + test("join operator selection") { + clearCache() + + Seq( + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]), + ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]), + ("SELECT * FROM testData JOIN testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + ("SELECT * FROM testData LEFT JOIN testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData RIGHT JOIN testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData FULL OUTER JOIN testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData LEFT JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + ("SELECT * FROM testData RIGHT JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + ("SELECT * FROM testData JOIN testData2 WHERE key > a", classOf[CartesianProduct]), + ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key > a", classOf[CartesianProduct]), + ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[HashOuterJoin]), + ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", classOf[HashOuterJoin]), - ("SELECT * FROM testData right join testData2 ON key = a and key=2", + ("SELECT * FROM testData right join testData2 ON key = a and key = 2", classOf[HashOuterJoin]), - ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]), - ("SELECT * FROM testData join testData2 ON key = a", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a and key=2", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key=2", classOf[ShuffledHashJoin]) - // TODO add BroadcastNestedLoopJoin - ) - cases1.foreach { c => assertJoin(c._1, c._2) } + ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]) + // TODO add BroadcastNestedLoopJoin + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } + + test("broadcasted hash join operator selection") { + clearCache() + sql("CACHE TABLE testData") + + Seq( + ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key = 2", classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key = 2", classOf[BroadcastHashJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + + sql("UNCACHE TABLE testData") } test("multiple-key equi-join is hash-join") { @@ -171,7 +178,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) - + checkAnswer( upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'n > 1)), (1, "A", null, null) :: @@ -180,7 +187,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) - + checkAnswer( upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'N > 1)), (1, "A", null, null) :: @@ -189,7 +196,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) - + checkAnswer( upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'l > 'L)), (1, "A", 1, "a") :: @@ -300,7 +307,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) - + checkAnswer( left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("left.N".attr !== 3))), (1, "A", null, null) :: @@ -310,7 +317,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) - + checkAnswer( left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("right.N".attr !== 3))), (1, "A", null, null) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 6c38575b13a2d..c4dd3e860f5fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -80,7 +80,7 @@ object TestData { UpperCaseData(3, "C") :: UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: - UpperCaseData(6, "F") :: Nil) + UpperCaseData(6, "F") :: Nil).toSchemaRDD upperCaseData.registerTempTable("upperCaseData") case class LowerCaseData(n: Int, l: String) @@ -89,7 +89,7 @@ object TestData { LowerCaseData(1, "a") :: LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: - LowerCaseData(4, "d") :: Nil) + LowerCaseData(4, "d") :: Nil).toSchemaRDD lowerCaseData.registerTempTable("lowerCaseData") case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) From bfa614b12795f1cfce4de0950f90cb8c4f2a7d53 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 27 Oct 2014 10:53:15 -0700 Subject: [PATCH 045/115] SPARK-4022 [CORE] [MLLIB] Replace colt dependency (LGPL) with commons-math This change replaces usages of colt with commons-math3 equivalents, and makes some minor necessary adjustments to related code and tests to match. Author: Sean Owen Closes #2928 from srowen/SPARK-4022 and squashes the following commits: 61a232f [Sean Owen] Fix failure due to different sampling in JavaAPISuite.sample() 16d66b8 [Sean Owen] Simplify seeding with call to reseedRandomGenerator a1a78e0 [Sean Owen] Use Well19937c 31c7641 [Sean Owen] Fix Python Poisson test by choosing a different seed; about 88% of seeds should work but 1 didn't, it seems 5c9c67f [Sean Owen] Additional test fixes from review d8f88e0 [Sean Owen] Replace colt with commons-math3. Some tests do not pass yet. --- LICENSE | 12 --- assembly/pom.xml | 4 + core/pom.xml | 6 -- .../apache/spark/partial/CountEvaluator.scala | 5 +- .../spark/partial/GroupedCountEvaluator.scala | 5 +- .../apache/spark/partial/MeanEvaluator.scala | 7 +- .../apache/spark/partial/StudentTCacher.scala | 9 +- .../apache/spark/partial/SumEvaluator.scala | 7 +- .../org/apache/spark/rdd/SampledRDD.scala | 9 +- .../spark/util/random/RandomSampler.scala | 10 +- .../util/random/StratifiedSamplingUtils.scala | 32 +++--- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../util/random/RandomSamplerSuite.scala | 9 +- examples/pom.xml | 8 ++ .../org/apache/spark/examples/LocalALS.scala | 97 ++++++++----------- .../org/apache/spark/examples/SparkALS.scala | 78 +++++++-------- mllib/pom.xml | 8 +- .../mllib/random/RandomDataGenerator.scala | 10 +- .../spark/mllib/stat/test/ChiSqTest.scala | 20 ++-- .../spark/mllib/tree/impl/BaggedPoint.scala | 8 +- .../spark/mllib/tree/RandomForestSuite.scala | 2 +- pom.xml | 6 -- python/pyspark/mllib/random.py | 2 +- 23 files changed, 175 insertions(+), 181 deletions(-) diff --git a/LICENSE b/LICENSE index a7eee041129cb..f1732fb47afc0 100644 --- a/LICENSE +++ b/LICENSE @@ -712,18 +712,6 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -======================================================================== -For colt: -======================================================================== - -Copyright (c) 1999 CERN - European Organization for Nuclear Research. -Permission to use, copy, modify, distribute and sell this software and its documentation for any purpose is hereby granted without fee, provided that the above copyright notice appear in all copies and that both that copyright notice and this permission notice appear in supporting documentation. CERN makes no representations about the suitability of this software for any purpose. It is provided "as is" without expressed or implied warranty. - -Packages hep.aida.* - -Written by Pavel Binko, Dino Ferrero Merlino, Wolfgang Hoschek, Tony Johnson, Andreas Pfeiffer, and others. Check the FreeHEP home page for more info. Permission to use and/or redistribute this work is granted under the terms of the LGPL License, with the exception that any usage related to military applications is expressly forbidden. The software and documentation made available under the terms of this license are provided with no warranty. - - ======================================================================== For SnapTree: ======================================================================== diff --git a/assembly/pom.xml b/assembly/pom.xml index bfef95b8deb95..11d4bea9361ab 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -146,6 +146,10 @@ com/google/common/base/Present* + + org.apache.commons.math3 + org.spark-project.commons.math3 + diff --git a/core/pom.xml b/core/pom.xml index 320d1076f7c03..5cd21e18e8ca7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -85,8 +85,6 @@ org.apache.commons commons-math3 - 3.3 - test com.google.code.findbugs @@ -162,10 +160,6 @@ json4s-jackson_${scala.binary.version} 3.2.10 - - colt - colt - org.apache.mesos mesos diff --git a/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala index 3155dfe165664..637492a97551b 100644 --- a/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.NormalDistribution /** * An ApproximateEvaluator for counts. @@ -46,7 +46,8 @@ private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double) val mean = (sum + 1 - p) / p val variance = (sum + 1) * (1 - p) / (p * p) val stdev = math.sqrt(variance) - val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2) + val confFactor = new NormalDistribution(). + inverseCumulativeProbability(1 - (1 - confidence) / 2) val low = mean - confFactor * stdev val high = mean + confFactor * stdev new BoundedDouble(mean, confidence, low, high) diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 8bb78123e3c9c..3ef3cc219dec6 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -24,7 +24,7 @@ import scala.collection.Map import scala.collection.mutable.HashMap import scala.reflect.ClassTag -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.NormalDistribution import org.apache.spark.util.collection.OpenHashMap @@ -55,7 +55,8 @@ private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, conf new HashMap[T, BoundedDouble] } else { val p = outputsMerged.toDouble / totalOutputs - val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2) + val confFactor = new NormalDistribution(). + inverseCumulativeProbability(1 - (1 - confidence) / 2) val result = new JHashMap[T, BoundedDouble](sums.size) sums.foreach { case (key, sum) => val mean = (sum + 1 - p) / p diff --git a/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala index d24959cba8727..787a21a61fdcf 100644 --- a/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.{NormalDistribution, TDistribution} import org.apache.spark.util.StatCounter @@ -45,9 +45,10 @@ private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double) val stdev = math.sqrt(counter.sampleVariance / counter.count) val confFactor = { if (counter.count > 100) { - Probability.normalInverse(1 - (1 - confidence) / 2) + new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) } else { - Probability.studentTInverse(1 - confidence, (counter.count - 1).toInt) + val degreesOfFreedom = (counter.count - 1).toInt + new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) } } val low = mean - confFactor * stdev diff --git a/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala index 92915ee66d29f..828bf96c2c0bd 100644 --- a/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala +++ b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.{TDistribution, NormalDistribution} /** * A utility class for caching Student's T distribution values for a given confidence level @@ -25,8 +25,10 @@ import cern.jet.stat.Probability * confidence intervals for many keys. */ private[spark] class StudentTCacher(confidence: Double) { + val NORMAL_APPROX_SAMPLE_SIZE = 100 // For samples bigger than this, use Gaussian approximation - val normalApprox = Probability.normalInverse(1 - (1 - confidence) / 2) + + val normalApprox = new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) val cache = Array.fill[Double](NORMAL_APPROX_SAMPLE_SIZE)(-1.0) def get(sampleSize: Long): Double = { @@ -35,7 +37,8 @@ private[spark] class StudentTCacher(confidence: Double) { } else { val size = sampleSize.toInt if (cache(size) < 0) { - cache(size) = Probability.studentTInverse(1 - confidence, size - 1) + val tDist = new TDistribution(size - 1) + cache(size) = tDist.inverseCumulativeProbability(1 - (1 - confidence) / 2) } cache(size) } diff --git a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala index d5336284571d2..1753c2561b678 100644 --- a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.{TDistribution, NormalDistribution} import org.apache.spark.util.StatCounter @@ -55,9 +55,10 @@ private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) val sumStdev = math.sqrt(sumVar) val confFactor = { if (counter.count > 100) { - Probability.normalInverse(1 - (1 - confidence) / 2) + new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) } else { - Probability.studentTInverse(1 - confidence, (counter.count - 1).toInt) + val degreesOfFreedom = (counter.count - 1).toInt + new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) } } val low = sumEstimate - confFactor * sumStdev diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index b097c30f8c231..9e8cee5331cf8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -21,8 +21,7 @@ import java.util.Random import scala.reflect.ClassTag -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.{Partition, TaskContext} @@ -53,9 +52,11 @@ private[spark] class SampledRDD[T: ClassTag]( if (withReplacement) { // For large datasets, the expected number of occurrences of each element in a sample with // replacement is Poisson(frac). We use that to get a count for each element. - val poisson = new Poisson(frac, new DRand(split.seed)) + val poisson = new PoissonDistribution(frac) + poisson.reseedRandomGenerator(split.seed) + firstParent[T].iterator(split.prev, context).flatMap { element => - val count = poisson.nextInt() + val count = poisson.sample() if (count == 0) { Iterator.empty // Avoid object allocation when we return 0 items, which is quite often } else { diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 32c5fdad75e58..ee389def20c8c 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -19,8 +19,7 @@ package org.apache.spark.util.random import java.util.Random -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.annotation.DeveloperApi @@ -87,15 +86,16 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) @DeveloperApi class PoissonSampler[T](mean: Double) extends RandomSampler[T, T] { - private[random] var rng = new Poisson(mean, new DRand) + private[random] var rng = new PoissonDistribution(mean) override def setSeed(seed: Long) { - rng = new Poisson(mean, new DRand(seed.toInt)) + rng = new PoissonDistribution(mean) + rng.reseedRandomGenerator(seed) } override def sample(items: Iterator[T]): Iterator[T] = { items.flatMap { item => - val count = rng.nextInt() + val count = rng.sample() if (count == 0) { Iterator.empty } else { diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index 8f95d7c6b799b..4fa357edd6f07 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -22,8 +22,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.Logging import org.apache.spark.SparkContext._ @@ -209,7 +208,7 @@ private[spark] object StratifiedSamplingUtils extends Logging { samplingRateByKey = computeThresholdByKey(finalResult, fractions) } (idx: Int, iter: Iterator[(K, V)]) => { - val rng = new RandomDataGenerator + val rng = new RandomDataGenerator() rng.reSeed(seed + idx) // Must use the same invoke pattern on the rng as in getSeqOp for without replacement // in order to generate the same sequence of random numbers when creating the sample @@ -245,9 +244,9 @@ private[spark] object StratifiedSamplingUtils extends Logging { // Must use the same invoke pattern on the rng as in getSeqOp for with replacement // in order to generate the same sequence of random numbers when creating the sample val copiesAccepted = if (acceptBound == 0) 0L else rng.nextPoisson(acceptBound) - val copiesWailisted = rng.nextPoisson(finalResult(key).waitListBound) + val copiesWaitlisted = rng.nextPoisson(finalResult(key).waitListBound) val copiesInSample = copiesAccepted + - (0 until copiesWailisted).count(i => rng.nextUniform() < thresholdByKey(key)) + (0 until copiesWaitlisted).count(i => rng.nextUniform() < thresholdByKey(key)) if (copiesInSample > 0) { Iterator.fill(copiesInSample.toInt)(item) } else { @@ -261,10 +260,10 @@ private[spark] object StratifiedSamplingUtils extends Logging { rng.reSeed(seed + idx) iter.flatMap { item => val count = rng.nextPoisson(fractions(item._1)) - if (count > 0) { - Iterator.fill(count)(item) - } else { + if (count == 0) { Iterator.empty + } else { + Iterator.fill(count)(item) } } } @@ -274,15 +273,24 @@ private[spark] object StratifiedSamplingUtils extends Logging { /** A random data generator that generates both uniform values and Poisson values. */ private class RandomDataGenerator { val uniform = new XORShiftRandom() - var poisson = new Poisson(1.0, new DRand) + // commons-math3 doesn't have a method to generate Poisson from an arbitrary mean; + // maintain a cache of Poisson(m) distributions for various m + val poissonCache = mutable.Map[Double, PoissonDistribution]() + var poissonSeed = 0L - def reSeed(seed: Long) { + def reSeed(seed: Long): Unit = { uniform.setSeed(seed) - poisson = new Poisson(1.0, new DRand(seed.toInt)) + poissonSeed = seed + poissonCache.clear() } def nextPoisson(mean: Double): Int = { - poisson.nextInt(mean) + val poisson = poissonCache.getOrElseUpdate(mean, { + val newPoisson = new PoissonDistribution(mean) + newPoisson.reseedRandomGenerator(poissonSeed) + newPoisson + }) + poisson.sample() } def nextUniform(): Double = { diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 814e40c4f77cc..0172876a264b8 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -142,7 +142,7 @@ public void sample() { JavaRDD rdd = sc.parallelize(ints); JavaRDD sample20 = rdd.sample(true, 0.2, 11); // expected 2 but of course result varies randomly a bit - Assert.assertEquals(3, sample20.count()); + Assert.assertEquals(1, sample20.count()); JavaRDD sample20NoReplacement = rdd.sample(false, 0.2, 11); Assert.assertEquals(2, sample20NoReplacement.count()); } diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index 36877476e708e..ba67d766a775a 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.util.random import java.util.Random -import cern.jet.random.Poisson +import org.apache.commons.math3.distribution.PoissonDistribution + import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar @@ -28,11 +29,11 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val a = List(1, 2, 3, 4, 5, 6, 7, 8, 9) var random: Random = _ - var poisson: Poisson = _ + var poisson: PoissonDistribution = _ before { random = mock[Random] - poisson = mock[Poisson] + poisson = mock[PoissonDistribution] } test("BernoulliSamplerWithRange") { @@ -101,7 +102,7 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar test("PoissonSampler") { expecting { for(x <- Seq(0, 1, 2, 0, 1, 1, 0, 0, 0)) { - poisson.nextInt().andReturn(x) + poisson.sample().andReturn(x) } } whenExecuting(poisson) { diff --git a/examples/pom.xml b/examples/pom.xml index eb49a0e5af22d..bc3291803c324 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -156,6 +156,10 @@ algebird-core_${scala.binary.version} 0.1.11 + + org.apache.commons + commons-math3 + org.scalatest scalatest_${scala.binary.version} @@ -268,6 +272,10 @@ com.google.common.base.Optional** + + org.apache.commons.math3 + org.spark-project.commons.math3 + diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 1f576319b3ca8..3d5259463003d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -17,11 +17,7 @@ package org.apache.spark.examples -import scala.math.sqrt - -import cern.colt.matrix._ -import cern.colt.matrix.linalg._ -import cern.jet.math._ +import org.apache.commons.math3.linear._ /** * Alternating least squares matrix factorization. @@ -30,84 +26,70 @@ import cern.jet.math._ * please refer to org.apache.spark.mllib.recommendation.ALS */ object LocalALS { + // Parameters set through command line arguments var M = 0 // Number of movies var U = 0 // Number of users var F = 0 // Number of features var ITERATIONS = 0 - val LAMBDA = 0.01 // Regularization coefficient - // Some COLT objects - val factory2D = DoubleFactory2D.dense - val factory1D = DoubleFactory1D.dense - val algebra = Algebra.DEFAULT - val blas = SeqBlas.seqBlas - - def generateR(): DoubleMatrix2D = { - val mh = factory2D.random(M, F) - val uh = factory2D.random(U, F) - algebra.mult(mh, algebra.transpose(uh)) + def generateR(): RealMatrix = { + val mh = randomMatrix(M, F) + val uh = randomMatrix(U, F) + mh.multiply(uh.transpose()) } - def rmse(targetR: DoubleMatrix2D, ms: Array[DoubleMatrix1D], - us: Array[DoubleMatrix1D]): Double = - { - val r = factory2D.make(M, U) + def rmse(targetR: RealMatrix, ms: Array[RealVector], us: Array[RealVector]): Double = { + val r = new Array2DRowRealMatrix(M, U) for (i <- 0 until M; j <- 0 until U) { - r.set(i, j, blas.ddot(ms(i), us(j))) + r.setEntry(i, j, ms(i).dotProduct(us(j))) } - blas.daxpy(-1, targetR, r) - val sumSqs = r.aggregate(Functions.plus, Functions.square) - sqrt(sumSqs / (M * U)) + val diffs = r.subtract(targetR) + var sumSqs = 0.0 + for (i <- 0 until M; j <- 0 until U) { + val diff = diffs.getEntry(i, j) + sumSqs += diff * diff + } + math.sqrt(sumSqs / (M.toDouble * U.toDouble)) } - def updateMovie(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D], - R: DoubleMatrix2D) : DoubleMatrix1D = - { - val XtX = factory2D.make(F, F) - val Xty = factory1D.make(F) + def updateMovie(i: Int, m: RealVector, us: Array[RealVector], R: RealMatrix) : RealVector = { + var XtX: RealMatrix = new Array2DRowRealMatrix(F, F) + var Xty: RealVector = new ArrayRealVector(F) // For each user that rated the movie for (j <- 0 until U) { val u = us(j) // Add u * u^t to XtX - blas.dger(1, u, u, XtX) + XtX = XtX.add(u.outerProduct(u)) // Add u * rating to Xty - blas.daxpy(R.get(i, j), u, Xty) + Xty = Xty.add(u.mapMultiply(R.getEntry(i, j))) } - // Add regularization coefs to diagonal terms + // Add regularization coefficients to diagonal terms for (d <- 0 until F) { - XtX.set(d, d, XtX.get(d, d) + LAMBDA * U) + XtX.addToEntry(d, d, LAMBDA * U) } // Solve it with Cholesky - val ch = new CholeskyDecomposition(XtX) - val Xty2D = factory2D.make(Xty.toArray, F) - val solved2D = ch.solve(Xty2D) - solved2D.viewColumn(0) + new CholeskyDecomposition(XtX).getSolver.solve(Xty) } - def updateUser(j: Int, u: DoubleMatrix1D, ms: Array[DoubleMatrix1D], - R: DoubleMatrix2D) : DoubleMatrix1D = - { - val XtX = factory2D.make(F, F) - val Xty = factory1D.make(F) + def updateUser(j: Int, u: RealVector, ms: Array[RealVector], R: RealMatrix) : RealVector = { + var XtX: RealMatrix = new Array2DRowRealMatrix(F, F) + var Xty: RealVector = new ArrayRealVector(F) // For each movie that the user rated for (i <- 0 until M) { val m = ms(i) // Add m * m^t to XtX - blas.dger(1, m, m, XtX) + XtX = XtX.add(m.outerProduct(m)) // Add m * rating to Xty - blas.daxpy(R.get(i, j), m, Xty) + Xty = Xty.add(m.mapMultiply(R.getEntry(i, j))) } - // Add regularization coefs to diagonal terms + // Add regularization coefficients to diagonal terms for (d <- 0 until F) { - XtX.set(d, d, XtX.get(d, d) + LAMBDA * M) + XtX.addToEntry(d, d, LAMBDA * M) } // Solve it with Cholesky - val ch = new CholeskyDecomposition(XtX) - val Xty2D = factory2D.make(Xty.toArray, F) - val solved2D = ch.solve(Xty2D) - solved2D.viewColumn(0) + new CholeskyDecomposition(XtX).getSolver.solve(Xty) } def showWarning() { @@ -135,21 +117,28 @@ object LocalALS { showWarning() - printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) + println(s"Running with M=$M, U=$U, F=$F, iters=$ITERATIONS") val R = generateR() // Initialize m and u randomly - var ms = Array.fill(M)(factory1D.random(F)) - var us = Array.fill(U)(factory1D.random(F)) + var ms = Array.fill(M)(randomVector(F)) + var us = Array.fill(U)(randomVector(F)) // Iteratively update movies then users for (iter <- 1 to ITERATIONS) { - println("Iteration " + iter + ":") + println(s"Iteration $iter:") ms = (0 until M).map(i => updateMovie(i, ms(i), us, R)).toArray us = (0 until U).map(j => updateUser(j, us(j), ms, R)).toArray println("RMSE = " + rmse(R, ms, us)) println() } } + + private def randomVector(n: Int): RealVector = + new ArrayRealVector(Array.fill(n)(math.random)) + + private def randomMatrix(rows: Int, cols: Int): RealMatrix = + new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random)) + } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index fde8ffeedf8b4..6c0ac8013ce34 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -17,11 +17,7 @@ package org.apache.spark.examples -import scala.math.sqrt - -import cern.colt.matrix._ -import cern.colt.matrix.linalg._ -import cern.jet.math._ +import org.apache.commons.math3.linear._ import org.apache.spark._ @@ -32,62 +28,53 @@ import org.apache.spark._ * please refer to org.apache.spark.mllib.recommendation.ALS */ object SparkALS { + // Parameters set through command line arguments var M = 0 // Number of movies var U = 0 // Number of users var F = 0 // Number of features var ITERATIONS = 0 - val LAMBDA = 0.01 // Regularization coefficient - // Some COLT objects - val factory2D = DoubleFactory2D.dense - val factory1D = DoubleFactory1D.dense - val algebra = Algebra.DEFAULT - val blas = SeqBlas.seqBlas - - def generateR(): DoubleMatrix2D = { - val mh = factory2D.random(M, F) - val uh = factory2D.random(U, F) - algebra.mult(mh, algebra.transpose(uh)) + def generateR(): RealMatrix = { + val mh = randomMatrix(M, F) + val uh = randomMatrix(U, F) + mh.multiply(uh.transpose()) } - def rmse(targetR: DoubleMatrix2D, ms: Array[DoubleMatrix1D], - us: Array[DoubleMatrix1D]): Double = - { - val r = factory2D.make(M, U) + def rmse(targetR: RealMatrix, ms: Array[RealVector], us: Array[RealVector]): Double = { + val r = new Array2DRowRealMatrix(M, U) for (i <- 0 until M; j <- 0 until U) { - r.set(i, j, blas.ddot(ms(i), us(j))) + r.setEntry(i, j, ms(i).dotProduct(us(j))) } - blas.daxpy(-1, targetR, r) - val sumSqs = r.aggregate(Functions.plus, Functions.square) - sqrt(sumSqs / (M * U)) + val diffs = r.subtract(targetR) + var sumSqs = 0.0 + for (i <- 0 until M; j <- 0 until U) { + val diff = diffs.getEntry(i, j) + sumSqs += diff * diff + } + math.sqrt(sumSqs / (M.toDouble * U.toDouble)) } - def update(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D], - R: DoubleMatrix2D) : DoubleMatrix1D = - { + def update(i: Int, m: RealVector, us: Array[RealVector], R: RealMatrix) : RealVector = { val U = us.size - val F = us(0).size - val XtX = factory2D.make(F, F) - val Xty = factory1D.make(F) + val F = us(0).getDimension + var XtX: RealMatrix = new Array2DRowRealMatrix(F, F) + var Xty: RealVector = new ArrayRealVector(F) // For each user that rated the movie for (j <- 0 until U) { val u = us(j) // Add u * u^t to XtX - blas.dger(1, u, u, XtX) + XtX = XtX.add(u.outerProduct(u)) // Add u * rating to Xty - blas.daxpy(R.get(i, j), u, Xty) + Xty = Xty.add(u.mapMultiply(R.getEntry(i, j))) } // Add regularization coefs to diagonal terms for (d <- 0 until F) { - XtX.set(d, d, XtX.get(d, d) + LAMBDA * U) + XtX.addToEntry(d, d, LAMBDA * U) } // Solve it with Cholesky - val ch = new CholeskyDecomposition(XtX) - val Xty2D = factory2D.make(Xty.toArray, F) - val solved2D = ch.solve(Xty2D) - solved2D.viewColumn(0) + new CholeskyDecomposition(XtX).getSolver.solve(Xty) } def showWarning() { @@ -118,7 +105,7 @@ object SparkALS { showWarning() - printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) + println(s"Running with M=$M, U=$U, F=$F, iters=$ITERATIONS") val sparkConf = new SparkConf().setAppName("SparkALS") val sc = new SparkContext(sparkConf) @@ -126,21 +113,21 @@ object SparkALS { val R = generateR() // Initialize m and u randomly - var ms = Array.fill(M)(factory1D.random(F)) - var us = Array.fill(U)(factory1D.random(F)) + var ms = Array.fill(M)(randomVector(F)) + var us = Array.fill(U)(randomVector(F)) // Iteratively update movies then users val Rc = sc.broadcast(R) var msb = sc.broadcast(ms) var usb = sc.broadcast(us) for (iter <- 1 to ITERATIONS) { - println("Iteration " + iter + ":") + println(s"Iteration $iter:") ms = sc.parallelize(0 until M, slices) .map(i => update(i, msb.value(i), usb.value, Rc.value)) .collect() msb = sc.broadcast(ms) // Re-broadcast ms because it was updated us = sc.parallelize(0 until U, slices) - .map(i => update(i, usb.value(i), msb.value, algebra.transpose(Rc.value))) + .map(i => update(i, usb.value(i), msb.value, Rc.value.transpose())) .collect() usb = sc.broadcast(us) // Re-broadcast us because it was updated println("RMSE = " + rmse(R, ms, us)) @@ -149,4 +136,11 @@ object SparkALS { sc.stop() } + + private def randomVector(n: Int): RealVector = + new ArrayRealVector(Array.fill(n)(math.random)) + + private def randomMatrix(rows: Int, cols: Int): RealMatrix = + new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random)) + } diff --git a/mllib/pom.xml b/mllib/pom.xml index 696e9396f627c..de062a4901596 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -65,12 +65,12 @@ junit junit - - org.apache.commons - commons-math3 - + + org.apache.commons + commons-math3 + org.scalatest scalatest_${scala.binary.version} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 28179fbc450c0..51f9b8657c640 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -17,8 +17,7 @@ package org.apache.spark.mllib.random -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} @@ -89,12 +88,13 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] { @DeveloperApi class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { - private var rng = new Poisson(mean, new DRand) + private var rng = new PoissonDistribution(mean) - override def nextValue(): Double = rng.nextDouble() + override def nextValue(): Double = rng.sample() override def setSeed(seed: Long) { - rng = new Poisson(mean, new DRand(seed.toInt)) + rng = new PoissonDistribution(mean) + rng.reseedRandomGenerator(seed) } override def copy(): PoissonGenerator = new PoissonGenerator(mean) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 0089419c2c5d4..ea82d39b72c03 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.stat.test import breeze.linalg.{DenseMatrix => BDM} -import cern.jet.stat.Probability.chiSquareComplemented +import org.apache.commons.math3.distribution.ChiSquaredDistribution import org.apache.spark.{SparkException, Logging} import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} @@ -33,7 +33,7 @@ import scala.collection.mutable * on an input of type `Matrix` in which independence between columns is assessed. * We also provide a method for computing the chi-squared statistic between each feature and the * label for an input `RDD[LabeledPoint]`, return an `Array[ChiSquaredTestResult]` of size = - * number of features in the inpuy RDD. + * number of features in the input RDD. * * Supported methods for goodness of fit: `pearson` (default) * Supported methods for independence: `pearson` (default) @@ -139,7 +139,7 @@ private[stat] object ChiSqTest extends Logging { } /* - * Pearon's goodness of fit test on the input observed and expected counts/relative frequencies. + * Pearson's goodness of fit test on the input observed and expected counts/relative frequencies. * Uniform distribution is assumed when `expected` is not passed in. */ def chiSquared(observed: Vector, @@ -188,12 +188,12 @@ private[stat] object ChiSqTest extends Logging { } } val df = size - 1 - val pValue = chiSquareComplemented(df, statistic) + val pValue = 1.0 - new ChiSquaredDistribution(df).cumulativeProbability(statistic) new ChiSqTestResult(pValue, df, statistic, PEARSON.name, NullHypothesis.goodnessOfFit.toString) } /* - * Pearon's independence test on the input contingency matrix. + * Pearson's independence test on the input contingency matrix. * TODO: optimize for SparseMatrix when it becomes supported. */ def chiSquaredMatrix(counts: Matrix, methodName:String = PEARSON.name): ChiSqTestResult = { @@ -238,7 +238,13 @@ private[stat] object ChiSqTest extends Logging { j += 1 } val df = (numCols - 1) * (numRows - 1) - val pValue = chiSquareComplemented(df, statistic) - new ChiSqTestResult(pValue, df, statistic, methodName, NullHypothesis.independence.toString) + if (df == 0) { + // 1 column or 1 row. Constant distribution is independent of anything. + // pValue = 1.0 and statistic = 0.0 in this case. + new ChiSqTestResult(1.0, 0, 0.0, methodName, NullHypothesis.independence.toString) + } else { + val pValue = 1.0 - new ChiSquaredDistribution(df).cumulativeProbability(statistic) + new ChiSqTestResult(pValue, df, statistic, methodName, NullHypothesis.independence.toString) + } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala index 937c8a2ac5836..e7a2127c5d2e7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala @@ -17,8 +17,7 @@ package org.apache.spark.mllib.tree.impl -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -60,12 +59,13 @@ private[tree] object BaggedPoint { input.mapPartitionsWithIndex { (partitionIndex, instances) => // TODO: Support different sampling rates, and sampling without replacement. // Use random seed = seed + partitionIndex + 1 to make generation reproducible. - val poisson = new Poisson(1.0, new DRand(seed + partitionIndex + 1)) + val poisson = new PoissonDistribution(1.0) + poisson.reseedRandomGenerator(seed + partitionIndex + 1) instances.map { instance => val subsampleWeights = new Array[Double](numSubsamples) var subsampleIndex = 0 while (subsampleIndex < numSubsamples) { - subsampleWeights(subsampleIndex) = poisson.nextInt() + subsampleWeights(subsampleIndex) = poisson.sample() subsampleIndex += 1 } new BaggedPoint(instance, subsampleWeights) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 6b13765b98f41..d3eff59aa0409 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -187,7 +187,7 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) val model = RandomForest.trainClassifier(input, strategy, numTrees = 2, featureSubsetStrategy = "sqrt", seed = 12345) - RandomForestSuite.validateClassifier(model, arr, 1.0) + RandomForestSuite.validateClassifier(model, arr, 0.0) } } diff --git a/pom.xml b/pom.xml index 2ebe1b8da588a..abcb97108c5d9 100644 --- a/pom.xml +++ b/pom.xml @@ -305,7 +305,6 @@ org.apache.commons commons-math3 3.3 - test com.google.code.findbugs @@ -431,11 +430,6 @@ akka-testkit_${scala.binary.version} ${akka.version} - - colt - colt - 1.2.0 - org.apache.mesos mesos diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 73baba4ace5f6..2202c51ab9c06 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -107,7 +107,7 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): distribution with the input mean. >>> mean = 100.0 - >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=1L) + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2L) >>> stats = x.stats() >>> stats.count() 1000L From 7e3a1ada86e6adf1ddd4d8a321824daf5f3b2c75 Mon Sep 17 00:00:00 2001 From: coderxiang Date: Mon, 27 Oct 2014 19:43:39 -0700 Subject: [PATCH 046/115] [MLlib] SPARK-3987: add test case on objective value for NNLS Also update step parameter to pass the proposed test Author: coderxiang Closes #2965 from coderxiang/nnls-test and squashes the following commits: 24b06f9 [coderxiang] add test case on objective value for NNLS; update step parameter to pass the test --- .../spark/mllib/optimization/NNLS.scala | 2 +- .../spark/mllib/optimization/NNLSSuite.scala | 30 +++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala index e4b436b023794..fef062e02b6ec 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala @@ -79,7 +79,7 @@ private[mllib] object NNLS { // stopping condition def stop(step: Double, ndir: Double, nx: Double): Boolean = { ((step.isNaN) // NaN - || (step < 1e-6) // too small or negative + || (step < 1e-7) // too small or negative || (step > 1e40) // too small; almost certainly numerical problems || (ndir < 1e-12 * nx) // gradient relatively too small || (ndir < 1e-32) // gradient absolutely too small; numerical issues may lurk diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala index b781a6aed9a8c..82c327bd49fcd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala @@ -37,6 +37,12 @@ class NNLSSuite extends FunSuite { (ata, atb) } + /** Compute the objective value */ + def computeObjectiveValue(ata: DoubleMatrix, atb: DoubleMatrix, x: DoubleMatrix): Double = { + val res = (x.transpose().mmul(ata).mmul(x)).mul(0.5).sub(atb.dot(x)) + res.get(0) + } + test("NNLS: exact solution cases") { val n = 20 val rand = new Random(12346) @@ -79,4 +85,28 @@ class NNLSSuite extends FunSuite { assert(x(i) >= 0) } } + + test("NNLS: objective value test") { + val n = 5 + val ata = new DoubleMatrix(5, 5 + , 517399.13534, 242529.67289, -153644.98976, 130802.84503, -798452.29283 + , 242529.67289, 126017.69765, -75944.21743, 81785.36128, -405290.60884 + , -153644.98976, -75944.21743, 46986.44577, -45401.12659, 247059.51049 + , 130802.84503, 81785.36128, -45401.12659, 67457.31310, -253747.03819 + , -798452.29283, -405290.60884, 247059.51049, -253747.03819, 1310939.40814 + ) + val atb = new DoubleMatrix(5, 1, + -31755.05710, 13047.14813, -20191.24443, 25993.77580, 11963.55017) + + /** reference solution obtained from matlab function quadprog */ + val refx = new DoubleMatrix(Array(34.90751, 103.96254, 0.00000, 27.82094, 58.79627)) + val refObj = computeObjectiveValue(ata, atb, refx) + + + val ws = NNLS.createWorkspace(n) + val x = new DoubleMatrix(NNLS.solve(ata, atb, ws)) + val obj = computeObjectiveValue(ata, atb, x) + + assert(obj < refObj + 1E-5) + } } From 418ad83fe113f2f90552eb7247670279b55aed28 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 27 Oct 2014 20:42:05 -0700 Subject: [PATCH 047/115] [SPARK-3911] [SQL] HiveSimpleUdf can not be optimized in constant folding ``` explain extended select cos(null) from src limit 1; ``` outputs: ``` Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5] MetastoreRelation default, src, None == Optimized Logical Plan == Limit 1 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5] MetastoreRelation default, src, None == Physical Plan == Limit 1 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5] HiveTableScan [], (MetastoreRelation default, src, None), None ``` After patching this PR it outputs ``` == Parsed Logical Plan == Limit 1 Project ['cos(null) AS c_0#0] UnresolvedRelation None, src, None == Analyzed Logical Plan == Limit 1 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#0] MetastoreRelation default, src, None == Optimized Logical Plan == Limit 1 Project [null AS c_0#0] MetastoreRelation default, src, None == Physical Plan == Limit 1 Project [null AS c_0#0] HiveTableScan [], (MetastoreRelation default, src, None), None ``` Author: Cheng Hao Closes #2771 from chenghao-intel/hive_udf_constant_folding and squashes the following commits: 1379c73 [Cheng Hao] duplicate the PlanTest with catalyst/plans/PlanTest 1e52dda [Cheng Hao] add unit test for hive simple udf constant folding 01609ff [Cheng Hao] support constant folding for HiveSimpleUdf --- .../spark/sql/catalyst/plans/PlanTest.scala | 3 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 10 ++++ .../org/apache/spark/sql/QueryTest.scala | 4 +- .../spark/sql/catalyst/plans/PlanTest.scala | 57 +++++++++++++++++++ .../sql/hive/execution/HivePlanTest.scala | 32 +++++++++++ 5 files changed, 104 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 7e9f47ef21df8..c4a1f899d8a13 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -33,7 +33,8 @@ class PlanTest extends FunSuite { * we must normalize them to check if two different queries are identical. */ protected def normalizeExprIds(plan: LogicalPlan) = { - val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min + val list = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)) + val minId = if (list.isEmpty) 0 else list.min plan transformAllExpressions { case a: AttributeReference => AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 68f93f247d9bb..683c820dec305 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -99,6 +99,16 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ @transient protected lazy val arguments = children.map(c => toInspector(c.dataType)).toArray + @transient + protected lazy val isUDFDeterministic = { + val udfType = function.getClass().getAnnotation(classOf[HiveUDFType]) + udfType != null && udfType.deterministic() + } + + override def foldable = { + isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable) + } + // Create parameter converters @transient protected lazy val conversionHelper = new ConversionHelper(method, arguments) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index 95921c3d7ae09..6b0641052052d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ @@ -29,7 +31,7 @@ import org.apache.spark.sql.catalyst.util._ * It is hard to have maven allow one subproject depend on another subprojects test code. * So, we duplicate this code here. */ -class QueryTest extends FunSuite { +class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[SchemaRDD]] to be executed diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala new file mode 100644 index 0000000000000..081d94b6fc020 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans + +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ +import org.scalatest.FunSuite + +/** + * *** DUPLICATED FROM sql/catalyst/plans. *** + * + * It is hard to have maven allow one subproject depend on another subprojects test code. + * So, we duplicate this code here. + */ +class PlanTest extends FunSuite { + + /** + * Since attribute references are given globally unique ids during analysis, + * we must normalize them to check if two different queries are identical. + */ + protected def normalizeExprIds(plan: LogicalPlan) = { + val list = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)) + val minId = if (list.isEmpty) 0 else list.min + plan transformAllExpressions { + case a: AttributeReference => + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) + } + } + + /** Fails the test if the two plans do not match */ + protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { + val normalized1 = normalizeExprIds(plan1) + val normalized2 = normalizeExprIds(plan2) + if (normalized1 != normalized2) + fail( + s""" + |== FAIL: Plans do not match === + |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + """.stripMargin) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala new file mode 100644 index 0000000000000..c939e6e99d28a --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala @@ -0,0 +1,32 @@ +/* + * 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.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive + +class HivePlanTest extends QueryTest { + import TestHive._ + + test("udf constant folding") { + val optimized = sql("SELECT cos(null) FROM src").queryExecution.optimizedPlan + val correctAnswer = sql("SELECT cast(null as double) FROM src").queryExecution.optimizedPlan + + comparePlans(optimized, correctAnswer) + } +} From 698a7eab7710cccdfd3a5234dc9572e4e674fff6 Mon Sep 17 00:00:00 2001 From: Alex Liu Date: Mon, 27 Oct 2014 20:43:29 -0700 Subject: [PATCH 048/115] [SPARK-3816][SQL] Add table properties from storage handler to output jobConf ...ob conf in SparkHadoopWriter class Author: Alex Liu Closes #2677 from alexliu68/SPARK-SQL-3816 and squashes the following commits: 79c269b [Alex Liu] [SPARK-3816][SQL] Add table properties from storage handler to job conf --- .../org/apache/spark/sql/hive/hiveWriterContainers.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 981ab954da489..bf2ce9df67c58 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ @@ -47,6 +48,13 @@ private[hive] class SparkHiveWriterContainer( with Serializable { private val now = new Date() + private val tableDesc: TableDesc = fileSinkConf.getTableInfo + // Add table properties from storage handler to jobConf, so any custom storage + // handler settings can be set to jobConf + if (tableDesc != null) { + PlanUtils.configureOutputJobPropertiesForStorageHandler(tableDesc) + Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) + } protected val conf = new SerializableWritable(jobConf) private var jobID = 0 From 89af6dfc3afb2b8fc60fa74afb52541dbf3c4e8f Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 27 Oct 2014 20:46:20 -0700 Subject: [PATCH 049/115] [SPARK-4041][SQL] Attributes names in table scan should converted to lowercase when compare with relation attributes In ```MetastoreRelation``` the attributes name is lowercase because of hive using lowercase for fields name, so we should convert attributes name in table scan lowercase in ```indexWhere(_.name == a.name)```. ```neededColumnIDs``` may be not correct if not convert to lowercase. Author: wangfei Author: scwf Closes #2884 from scwf/fixColumnIds and squashes the following commits: 6174046 [scwf] use AttributeMap for this issue dc74a24 [wangfei] use lowerName and add a test case for this issue 3ff3a80 [wangfei] more safer change 294fcb7 [scwf] attributes names in table scan should convert lowercase in neededColumnsIDs --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 6 ++++++ .../apache/spark/sql/hive/execution/HiveTableScan.scala | 9 +++++---- .../spark/sql/hive/execution/HiveTableScanSuite.scala | 9 +++++++++ 3 files changed, 20 insertions(+), 4 deletions(-) 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 04c48c385966e..39d87a9d14fa7 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 @@ -307,4 +307,10 @@ private[hive] case class MetastoreRelation val attributes = hiveQlTable.getCols.map(_.toAttribute) val output = attributes ++ partitionKeys + + /** An attribute map that can be used to lookup original attributes based on expression id. */ + val attributeMap = AttributeMap(output.map(o => (o,o))) + + /** An attribute map for determining the ordinal for non-partition columns. */ + val columnOrdinals = AttributeMap(attributes.zipWithIndex) } 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 b7f3ade4ea819..d39413a44a6cb 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 @@ -44,7 +44,7 @@ import org.apache.spark.sql.hive._ */ @DeveloperApi case class HiveTableScan( - attributes: Seq[Attribute], + requestedAttributes: Seq[Attribute], relation: MetastoreRelation, partitionPruningPred: Option[Expression])( @transient val context: HiveContext) @@ -53,6 +53,9 @@ case class HiveTableScan( require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, "Partition pruning predicates only supported for partitioned tables.") + // Retrieve the original attributes based on expression ID so that capitalization matches. + val attributes = requestedAttributes.map(relation.attributeMap) + // Bind all partition key attribute references in the partition pruning predicate for later // evaluation. private[this] val boundPruningPred = partitionPruningPred.map { pred => @@ -81,9 +84,7 @@ case class HiveTableScan( private def addColumnMetadataToConf(hiveConf: HiveConf) { // Specifies needed column IDs for those non-partitioning columns. - val neededColumnIDs = - attributes.map(a => - relation.attributes.indexWhere(_.name == a.name): Integer).filter(index => index >= 0) + val neededColumnIDs = attributes.flatMap(relation.columnOrdinals.get).map(o => o: Integer) HiveShim.appendReadColumns(hiveConf, neededColumnIDs, attributes.map(_.name)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index c5736723b47c0..2f3db95882093 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import org.apache.spark.sql.hive.test.TestHive + class HiveTableScanSuite extends HiveComparisonTest { createQueryTest("partition_based_table_scan_with_different_serde", @@ -38,4 +40,11 @@ class HiveTableScanSuite extends HiveComparisonTest { | |SELECT * from part_scan_test; """.stripMargin) + + test("Spark-4041: lowercase issue") { + TestHive.sql("CREATE TABLE tb (KEY INT, VALUE STRING) STORED AS ORC") + TestHive.sql("insert into table tb select key, value from src") + TestHive.sql("select KEY from tb where VALUE='just_for_test' limit 5").collect() + TestHive.sql("drop table tb") + } } From 27470d3406bc0adde3da79ca34ebf9bc512514b6 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 27 Oct 2014 20:50:09 -0700 Subject: [PATCH 050/115] [SQL] Correct a variable name in JavaApplySchemaSuite.applySchemaToJSON `schemaRDD2` is not tested because `schemaRDD1` is registered again. Author: Yin Huai Closes #2869 from yhuai/JavaApplySchemaSuite and squashes the following commits: 95fe894 [Yin Huai] Correct variable name. --- .../org/apache/spark/sql/api/java/JavaApplySchemaSuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java index 33e5020bc636a..9435a88009a5f 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -156,7 +156,7 @@ public void applySchemaToJSON() { JavaSchemaRDD schemaRDD2 = javaSqlCtx.jsonRDD(jsonRDD, expectedSchema); StructType actualSchema2 = schemaRDD2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); - schemaRDD1.registerTempTable("jsonTable2"); + schemaRDD2.registerTempTable("jsonTable2"); List actual2 = javaSqlCtx.sql("select * from jsonTable2").collect(); Assert.assertEquals(expectedResult, actual2); } From 0c34fa5b4b3c1c20d7a2d7df3a8ae757b532dd32 Mon Sep 17 00:00:00 2001 From: wangxiaojing Date: Mon, 27 Oct 2014 22:00:36 -0700 Subject: [PATCH 051/115] [SPARK-3907][SQL] Add truncate table support JIRA issue: [SPARK-3907]https://issues.apache.org/jira/browse/SPARK-3907 Add turncate table support TRUNCATE TABLE table_name [PARTITION partition_spec]; partition_spec: : (partition_col = partition_col_value, partition_col = partiton_col_value, ...) Removes all rows from a table or partition(s). Currently target table should be native/managed table or exception will be thrown. User can specify partial partition_spec for truncating multiple partitions at once and omitting partition_spec will truncate all partitions in the table. Author: wangxiaojing Closes #2770 from wangxiaojing/spark-3907 and squashes the following commits: 63dbd81 [wangxiaojing] change hive scalastyle 7a03707 [wangxiaojing] add comment f6e710e [wangxiaojing] change truncate table a1f692c [wangxiaojing] Correct spelling mistakes 3b20007 [wangxiaojing] add truncate can not support column err message e483547 [wangxiaojing] add golden file 77b1f20 [wangxiaojing] add truncate table support --- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 1 + .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 7 ++++++- .../truncate_column-0-616cad77ad5e7ac74da0d7425a7869a | 0 .../truncate_column-1-418ec894d08c33fd712eb358f579b7a0 | 1 + ...runcate_column_merge-0-46e8cc1556fa8586802a26267a906acf | 0 .../truncate_table-0-d16efe9bac079f0c5fc6cc424a8fa3eb | 0 .../truncate_table-1-ec0e3744208003f18c33a1f2c4c1e2c6 | 0 .../truncate_table-10-9ba46fdca3f0f4da8991cb5c7b01efdb | 0 .../truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 | 1 + .../truncate_table-12-7dee32ebe9887833a9ae2ea6e5568028 | 0 .../truncate_table-13-3230cfbe1871330193c3190c77582fe | 0 .../truncate_table-14-ae23925663d7e9b7e97c42b66086d835 | 0 .../truncate_table-15-7850dc059f9d00eb9439d477e92cb913 | 0 .../truncate_table-16-623e41aa678d5abc8341a8cee0ac8f94 | 0 .../truncate_table-17-8c71d29e7db6a8d1cb5746458c7741e6 | 0 .../truncate_table-18-64d431f93d8a44fb143cb4b87d63a105 | 0 .../truncate_table-19-1325d566d66f21a06543271c73a95a6f | 0 .../truncate_table-2-fc4118284bf8301cf0d1056c388f963a | 0 .../truncate_table-20-91f869cc79191b87d31cfd0eca2839f4 | 0 .../truncate_table-21-f635675d59df31843e7be41af7b9e4fa | 0 .../truncate_table-22-f121fdc101603a8220c0f18e867f581e | 0 .../truncate_table-23-63988ac685a3bd645787116353f024d2 | 0 .../truncate_table-3-ecca1d24f36175932911a6e7a78ece2d | 0 .../truncate_table-4-88e636ed8bdf647a02ff269aa3ebfe62 | 0 .../truncate_table-5-42aeecc67917d731e60fc46bde021d49 | 0 .../truncate_table-6-5a6776344f711298f27a8f1d3b47d107 | 0 .../truncate_table-7-65e270fb0b61886aa85255d77eb65794 | 0 .../truncate_table-8-e7699db3640f3b9b1fe44d6b8c9b507e | 0 .../truncate_table-9-eedfbb9479ac6c1b955b8e9b41994da4 | 0 29 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/truncate_column-0-616cad77ad5e7ac74da0d7425a7869a create mode 100644 sql/hive/src/test/resources/golden/truncate_column-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/truncate_column_merge-0-46e8cc1556fa8586802a26267a906acf create mode 100644 sql/hive/src/test/resources/golden/truncate_table-0-d16efe9bac079f0c5fc6cc424a8fa3eb create mode 100644 sql/hive/src/test/resources/golden/truncate_table-1-ec0e3744208003f18c33a1f2c4c1e2c6 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-10-9ba46fdca3f0f4da8991cb5c7b01efdb create mode 100644 sql/hive/src/test/resources/golden/truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-12-7dee32ebe9887833a9ae2ea6e5568028 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-13-3230cfbe1871330193c3190c77582fe create mode 100644 sql/hive/src/test/resources/golden/truncate_table-14-ae23925663d7e9b7e97c42b66086d835 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-15-7850dc059f9d00eb9439d477e92cb913 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-16-623e41aa678d5abc8341a8cee0ac8f94 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-17-8c71d29e7db6a8d1cb5746458c7741e6 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-18-64d431f93d8a44fb143cb4b87d63a105 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-19-1325d566d66f21a06543271c73a95a6f create mode 100644 sql/hive/src/test/resources/golden/truncate_table-2-fc4118284bf8301cf0d1056c388f963a create mode 100644 sql/hive/src/test/resources/golden/truncate_table-20-91f869cc79191b87d31cfd0eca2839f4 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-21-f635675d59df31843e7be41af7b9e4fa create mode 100644 sql/hive/src/test/resources/golden/truncate_table-22-f121fdc101603a8220c0f18e867f581e create mode 100644 sql/hive/src/test/resources/golden/truncate_table-23-63988ac685a3bd645787116353f024d2 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-3-ecca1d24f36175932911a6e7a78ece2d create mode 100644 sql/hive/src/test/resources/golden/truncate_table-4-88e636ed8bdf647a02ff269aa3ebfe62 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-5-42aeecc67917d731e60fc46bde021d49 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-6-5a6776344f711298f27a8f1d3b47d107 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-7-65e270fb0b61886aa85255d77eb65794 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-8-e7699db3640f3b9b1fe44d6b8c9b507e create mode 100644 sql/hive/src/test/resources/golden/truncate_table-9-eedfbb9479ac6c1b955b8e9b41994da4 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 15cd62d3bf869..4fc26d6f55511 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 @@ -767,6 +767,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "touch", "transform_ppr1", "transform_ppr2", + "truncate_table", "type_cast_1", "type_widening", "udaf_collect_set", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 54c619722ee12..aa80b2f04de25 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -124,7 +124,8 @@ private[hive] object HiveQl { // Commands that we do not need to explain. protected val noExplainCommands = Seq( "TOK_CREATETABLE", - "TOK_DESCTABLE" + "TOK_DESCTABLE", + "TOK_TRUNCATETABLE" // truncate table" is a NativeCommand, does not need to explain. ) ++ nativeCommands protected val hqlParser = { @@ -490,6 +491,10 @@ private[hive] object HiveQl { // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder + // Support "TRUNCATE TABLE table_name [PARTITION partition_spec]" + case Token("TOK_TRUNCATETABLE", + Token("TOK_TABLE_PARTITION",table)::Nil) => NativePlaceholder + case Token("TOK_QUERY", Token("TOK_FROM", fromClause :: Nil) :: insertClauses) => diff --git a/sql/hive/src/test/resources/golden/truncate_column-0-616cad77ad5e7ac74da0d7425a7869a b/sql/hive/src/test/resources/golden/truncate_column-0-616cad77ad5e7ac74da0d7425a7869a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_column-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/truncate_column-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/truncate_column-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/truncate_column_merge-0-46e8cc1556fa8586802a26267a906acf b/sql/hive/src/test/resources/golden/truncate_column_merge-0-46e8cc1556fa8586802a26267a906acf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-0-d16efe9bac079f0c5fc6cc424a8fa3eb b/sql/hive/src/test/resources/golden/truncate_table-0-d16efe9bac079f0c5fc6cc424a8fa3eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-1-ec0e3744208003f18c33a1f2c4c1e2c6 b/sql/hive/src/test/resources/golden/truncate_table-1-ec0e3744208003f18c33a1f2c4c1e2c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-10-9ba46fdca3f0f4da8991cb5c7b01efdb b/sql/hive/src/test/resources/golden/truncate_table-10-9ba46fdca3f0f4da8991cb5c7b01efdb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 b/sql/hive/src/test/resources/golden/truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/truncate_table-12-7dee32ebe9887833a9ae2ea6e5568028 b/sql/hive/src/test/resources/golden/truncate_table-12-7dee32ebe9887833a9ae2ea6e5568028 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-13-3230cfbe1871330193c3190c77582fe b/sql/hive/src/test/resources/golden/truncate_table-13-3230cfbe1871330193c3190c77582fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-14-ae23925663d7e9b7e97c42b66086d835 b/sql/hive/src/test/resources/golden/truncate_table-14-ae23925663d7e9b7e97c42b66086d835 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-15-7850dc059f9d00eb9439d477e92cb913 b/sql/hive/src/test/resources/golden/truncate_table-15-7850dc059f9d00eb9439d477e92cb913 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-16-623e41aa678d5abc8341a8cee0ac8f94 b/sql/hive/src/test/resources/golden/truncate_table-16-623e41aa678d5abc8341a8cee0ac8f94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-17-8c71d29e7db6a8d1cb5746458c7741e6 b/sql/hive/src/test/resources/golden/truncate_table-17-8c71d29e7db6a8d1cb5746458c7741e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-18-64d431f93d8a44fb143cb4b87d63a105 b/sql/hive/src/test/resources/golden/truncate_table-18-64d431f93d8a44fb143cb4b87d63a105 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-19-1325d566d66f21a06543271c73a95a6f b/sql/hive/src/test/resources/golden/truncate_table-19-1325d566d66f21a06543271c73a95a6f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-2-fc4118284bf8301cf0d1056c388f963a b/sql/hive/src/test/resources/golden/truncate_table-2-fc4118284bf8301cf0d1056c388f963a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-20-91f869cc79191b87d31cfd0eca2839f4 b/sql/hive/src/test/resources/golden/truncate_table-20-91f869cc79191b87d31cfd0eca2839f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-21-f635675d59df31843e7be41af7b9e4fa b/sql/hive/src/test/resources/golden/truncate_table-21-f635675d59df31843e7be41af7b9e4fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-22-f121fdc101603a8220c0f18e867f581e b/sql/hive/src/test/resources/golden/truncate_table-22-f121fdc101603a8220c0f18e867f581e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-23-63988ac685a3bd645787116353f024d2 b/sql/hive/src/test/resources/golden/truncate_table-23-63988ac685a3bd645787116353f024d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-3-ecca1d24f36175932911a6e7a78ece2d b/sql/hive/src/test/resources/golden/truncate_table-3-ecca1d24f36175932911a6e7a78ece2d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-4-88e636ed8bdf647a02ff269aa3ebfe62 b/sql/hive/src/test/resources/golden/truncate_table-4-88e636ed8bdf647a02ff269aa3ebfe62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-5-42aeecc67917d731e60fc46bde021d49 b/sql/hive/src/test/resources/golden/truncate_table-5-42aeecc67917d731e60fc46bde021d49 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-6-5a6776344f711298f27a8f1d3b47d107 b/sql/hive/src/test/resources/golden/truncate_table-6-5a6776344f711298f27a8f1d3b47d107 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-7-65e270fb0b61886aa85255d77eb65794 b/sql/hive/src/test/resources/golden/truncate_table-7-65e270fb0b61886aa85255d77eb65794 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-8-e7699db3640f3b9b1fe44d6b8c9b507e b/sql/hive/src/test/resources/golden/truncate_table-8-e7699db3640f3b9b1fe44d6b8c9b507e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-9-eedfbb9479ac6c1b955b8e9b41994da4 b/sql/hive/src/test/resources/golden/truncate_table-9-eedfbb9479ac6c1b955b8e9b41994da4 new file mode 100644 index 0000000000000..e69de29bb2d1d From 7c0c26cd1241e1fde3c6f1f659a43b9c40ee3d42 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Mon, 27 Oct 2014 23:31:46 -0700 Subject: [PATCH 052/115] [SPARK-4064]NioBlockTransferService.fetchBlocks may cause spark to hang. cc @rxin Author: GuoQiang Li Closes #2929 from witgo/SPARK-4064 and squashes the following commits: 20110f2 [GuoQiang Li] Modify the exception msg 3425225 [GuoQiang Li] review commits 2b07e49 [GuoQiang Li] If we create a lot of big broadcast variables, Spark may hang --- .../network/nio/NioBlockTransferService.scala | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index 5add4fc433fb3..e3113205bebdc 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -95,16 +95,21 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa future.onSuccess { case message => val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - listener.onBlockFetchFailure( - new SparkException(s"Unexpected message ${blockMessage.getType} received from $cmId")) - } else { - val blockId = blockMessage.getId - val networkSize = blockMessage.getData.limit() - listener.onBlockFetchSuccess( - blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + // SPARK-4064: In some cases(eg. Remote block was removed) blockMessageArray may be empty. + if (blockMessageArray.isEmpty) { + listener.onBlockFetchFailure( + new SparkException(s"Received empty message from $cmId")) + } else { + for (blockMessage <- blockMessageArray) { + val msgType = blockMessage.getType + if (msgType != BlockMessage.TYPE_GOT_BLOCK) { + listener.onBlockFetchFailure( + new SparkException(s"Unexpected message ${msgType} received from $cmId")) + } else { + val blockId = blockMessage.getId + listener.onBlockFetchSuccess( + blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + } } } }(cm.futureExecContext) From 4ceb048b38949dd0a909d2ee6777607341c9c93a Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Mon, 27 Oct 2014 23:55:13 -0700 Subject: [PATCH 053/115] fix broken links in README.md seems like `building-spark.html` was renamed to `building-with-maven.html`? Is Maven the blessed build tool these days, or SBT? I couldn't find a building-with-sbt page so I went with the Maven one here. Author: Ryan Williams Closes #2859 from ryan-williams/broken-links-readme and squashes the following commits: 7692253 [Ryan Williams] fix broken links in README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index dbf53dcd76b2d..9916ac7b1ae8e 100644 --- a/README.md +++ b/README.md @@ -84,7 +84,7 @@ storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs. Please refer to the build documentation at -["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-spark.html#specifying-the-hadoop-version) +["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-with-maven.html#specifying-the-hadoop-version) for detailed guidance on building for a particular distribution of Hadoop, including building for particular Hive and Hive Thriftserver distributions. See also ["Third Party Hadoop Distributions"](http://spark.apache.org/docs/latest/hadoop-third-party-distributions.html) From 46c63417c1bb1aea07baf9036cc5b8f1c3781bbe Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 28 Oct 2014 00:04:16 -0700 Subject: [PATCH 054/115] [SPARK-4107] Fix incorrect handling of read() and skip() return values `read()` may return fewer bytes than requested; when this occurred, the old code would silently return less data than requested, which might cause stream corruption errors. `skip()` faces similar issues, too. This patch fixes several cases where we mis-handle these methods' return values. Author: Josh Rosen Closes #2969 from JoshRosen/file-channel-read-fix and squashes the following commits: e724a9f [Josh Rosen] Fix similar issue of not checking skip() return value. cbc03ce [Josh Rosen] Update the other log message, too. 01e6015 [Josh Rosen] file.getName -> file.getAbsolutePath d961d95 [Josh Rosen] Fix another issue in FileServerSuite. b9265d2 [Josh Rosen] Fix a similar (minor) issue in TestUtils. cd9d76f [Josh Rosen] Fix a similar error in Tachyon: 3db0008 [Josh Rosen] Fix a similar read() error in Utils.offsetBytes(). db985ed [Josh Rosen] Fix unsafe usage of FileChannel.read(): --- .../scala/org/apache/spark/TestUtils.scala | 9 ++------ .../apache/spark/network/ManagedBuffer.scala | 10 +++++++-- .../shuffle/IndexShuffleBlockManager.scala | 4 +++- .../org/apache/spark/storage/DiskStore.scala | 10 +++++++-- .../apache/spark/storage/TachyonStore.scala | 21 +++++++------------ .../scala/org/apache/spark/util/Utils.scala | 6 +++--- .../org/apache/spark/FileServerSuite.scala | 8 ++----- 7 files changed, 33 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index e72826dc25f41..34078142f5385 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -23,8 +23,8 @@ import java.util.jar.{JarEntry, JarOutputStream} import scala.collection.JavaConversions._ +import com.google.common.io.{ByteStreams, Files} import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} -import com.google.common.io.Files import org.apache.spark.util.Utils @@ -64,12 +64,7 @@ private[spark] object TestUtils { jarStream.putNextEntry(jarEntry) val in = new FileInputStream(file) - val buffer = new Array[Byte](10240) - var nRead = 0 - while (nRead <= 0) { - nRead = in.read(buffer, 0, buffer.length) - jarStream.write(buffer, 0, nRead) - } + ByteStreams.copy(in, jarStream) in.close() } jarStream.close() diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala index 4c9ca97a2a6b7..4211ba4e43b9e 100644 --- a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -81,7 +81,13 @@ final class FileSegmentManagedBuffer(val file: File, val offset: Long, val lengt // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. if (length < MIN_MEMORY_MAP_BYTES) { val buf = ByteBuffer.allocate(length.toInt) - channel.read(buf, offset) + channel.position(offset) + while (buf.remaining() != 0) { + if (channel.read(buf) == -1) { + throw new IOException("Reached EOF before filling buffer\n" + + s"offset=$offset\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") + } + } buf.flip() buf } else { @@ -106,7 +112,7 @@ final class FileSegmentManagedBuffer(val file: File, val offset: Long, val lengt var is: FileInputStream = null try { is = new FileInputStream(file) - is.skip(offset) + ByteStreams.skipFully(is, offset) ByteStreams.limit(is, length) } catch { case e: IOException => diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 4ab34336d3f01..b5cd34cacd520 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -20,6 +20,8 @@ package org.apache.spark.shuffle import java.io._ import java.nio.ByteBuffer +import com.google.common.io.ByteStreams + import org.apache.spark.SparkEnv import org.apache.spark.network.{ManagedBuffer, FileSegmentManagedBuffer} import org.apache.spark.storage._ @@ -101,7 +103,7 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { val in = new DataInputStream(new FileInputStream(indexFile)) try { - in.skip(blockId.reduceId * 8) + ByteStreams.skipFully(in, blockId.reduceId * 8) val offset = in.readLong() val nextOffset = in.readLong() new FileSegmentManagedBuffer( diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index bac459e835a3f..8dadf6794039e 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{File, FileOutputStream, RandomAccessFile} +import java.io.{IOException, File, FileOutputStream, RandomAccessFile} import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode @@ -110,7 +110,13 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc // For small files, directly read rather than memory map if (length < minMemoryMapBytes) { val buf = ByteBuffer.allocate(length.toInt) - channel.read(buf, offset) + channel.position(offset) + while (buf.remaining() != 0) { + if (channel.read(buf) == -1) { + throw new IOException("Reached EOF before filling buffer\n" + + s"offset=$offset\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") + } + } buf.flip() Some(buf) } else { diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index 932b5616043b4..6dbad5ff0518e 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.IOException import java.nio.ByteBuffer +import com.google.common.io.ByteStreams import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging @@ -105,25 +106,17 @@ private[spark] class TachyonStore( return None } val is = file.getInStream(ReadType.CACHE) - var buffer: ByteBuffer = null + assert (is != null) try { - if (is != null) { - val size = file.length - val bs = new Array[Byte](size.asInstanceOf[Int]) - val fetchSize = is.read(bs, 0, size.asInstanceOf[Int]) - buffer = ByteBuffer.wrap(bs) - if (fetchSize != size) { - logWarning(s"Failed to fetch the block $blockId from Tachyon: Size $size " + - s"is not equal to fetched size $fetchSize") - return None - } - } + val size = file.length + val bs = new Array[Byte](size.asInstanceOf[Int]) + ByteStreams.readFully(is, bs) + Some(ByteBuffer.wrap(bs)) } catch { case ioe: IOException => logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe) - return None + None } - Some(buffer) } override def contains(blockId: BlockId): Boolean = { 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 93ac9f1c33ea8..4660030155854 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -33,7 +33,7 @@ import scala.reflect.ClassTag import scala.util.Try import scala.util.control.{ControlThrowable, NonFatal} -import com.google.common.io.Files +import com.google.common.io.{ByteStreams, Files} import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration @@ -1062,8 +1062,8 @@ private[spark] object Utils extends Logging { val stream = new FileInputStream(file) try { - stream.skip(effectiveStart) - stream.read(buff) + ByteStreams.skipFully(stream, effectiveStart) + ByteStreams.readFully(stream, buff) } finally { stream.close() } diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index a8867020e457d..379c2a6ea4b55 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark import java.io._ import java.util.jar.{JarEntry, JarOutputStream} +import com.google.common.io.ByteStreams import org.scalatest.FunSuite import org.apache.spark.SparkContext._ @@ -58,12 +59,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { jar.putNextEntry(jarEntry) val in = new FileInputStream(textFile) - val buffer = new Array[Byte](10240) - var nRead = 0 - while (nRead <= 0) { - nRead = in.read(buffer, 0, buffer.length) - jar.write(buffer, 0, nRead) - } + ByteStreams.copy(in, jar) in.close() jar.close() From fae095bc7c4097859af522ced77f09cf6be17691 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 28 Oct 2014 03:50:22 -0700 Subject: [PATCH 055/115] [SPARK-3961] [MLlib] [PySpark] Python API for mllib.feature Added completed Python API for MLlib.feature Normalizer StandardScalerModel StandardScaler HashTF IDFModel IDF cc mengxr Author: Davies Liu Author: Davies Liu Closes #2819 from davies/feature and squashes the following commits: 4f48f48 [Davies Liu] add a note for HashingTF 67f6d21 [Davies Liu] address comments b628693 [Davies Liu] rollback changes in Word2Vec efb4f4f [Davies Liu] Merge branch 'master' into feature 806c7c2 [Davies Liu] address comments 3abb8c2 [Davies Liu] address comments 59781b9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into feature a405ae7 [Davies Liu] fix tests 7a1891a [Davies Liu] fix tests 486795f [Davies Liu] update programming guide, HashTF -> HashingTF 8a50584 [Davies Liu] Python API for mllib.feature --- docs/mllib-feature-extraction.md | 85 ++++ .../mllib/api/python/PythonMLLibAPI.scala | 49 ++- .../mllib/feature/VectorTransformer.scala | 11 + .../apache/spark/mllib/feature/Word2Vec.scala | 4 +- python/pyspark/mllib/feature.py | 395 +++++++++++++++--- python/pyspark/mllib/linalg.py | 16 +- 6 files changed, 499 insertions(+), 61 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 11622414494e4..886d71df474bc 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -95,8 +95,49 @@ tf.cache() val idf = new IDF(minDocFreq = 2).fit(tf) val tfidf: RDD[Vector] = idf.transform(tf) {% endhighlight %} + +

+ +TF and IDF are implemented in [HashingTF](api/python/pyspark.mllib.html#pyspark.mllib.feature.HashingTF) +and [IDF](api/python/pyspark.mllib.html#pyspark.mllib.feature.IDF). +`HashingTF` takes an RDD of list as the input. +Each record could be an iterable of strings or other types. + +{% highlight python %} +from pyspark import SparkContext +from pyspark.mllib.feature import HashingTF + +sc = SparkContext() +# Load documents (one per line). +documents = sc.textFile("...").map(lambda line: line.split(" ")) + +hashingTF = HashingTF() +tf = hashingTF.transform(documents) +{% endhighlight %} + +While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: +first to compute the IDF vector and second to scale the term frequencies by IDF. + +{% highlight python %} +from pyspark.mllib.feature import IDF + +# ... continue from the previous example +tf.cache() +idf = IDF().fit(tf) +tfidf = idf.transform(tf) +{% endhighlight %} + +MLLib's IDF implementation provides an option for ignoring terms which occur in less than a +minimum number of documents. In such cases, the IDF for these terms is set to 0. This feature +can be used by passing the `minDocFreq` value to the IDF constructor. +{% highlight python %} +# ... continue from the previous example +tf.cache() +idf = IDF(minDocFreq=2).fit(tf) +tfidf = idf.transform(tf) +{% endhighlight %}
@@ -223,6 +264,29 @@ val data1 = data.map(x => (x.label, scaler1.transform(x.features))) val data2 = data.map(x => (x.label, scaler2.transform(Vectors.dense(x.features.toArray)))) {% endhighlight %} + +
+{% highlight python %} +from pyspark.mllib.util import MLUtils +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.feature import StandardScaler + +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +label = data.map(lambda x: x.label) +features = data.map(lambda x: x.features) + +scaler1 = StandardScaler().fit(features) +scaler2 = StandardScaler(withMean=True, withStd=True).fit(features) + +# data1 will be unit variance. +data1 = label.zip(scaler1.transform(features)) + +# Without converting the features into dense vectors, transformation with zero mean will raise +# exception on sparse vector. +# data2 will be unit variance and zero mean. +data2 = label.zip(scaler1.transform(features.map(lambda x: Vectors.dense(x.toArray())))) +{% endhighlight %} +
## Normalizer @@ -267,4 +331,25 @@ val data1 = data.map(x => (x.label, normalizer1.transform(x.features))) val data2 = data.map(x => (x.label, normalizer2.transform(x.features))) {% endhighlight %} + +
+{% highlight python %} +from pyspark.mllib.util import MLUtils +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.feature import Normalizer + +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +labels = data.map(lambda x: x.label) +features = data.map(lambda x: x.features) + +normalizer1 = Normalizer() +normalizer2 = Normalizer(p=float("inf")) + +# Each sample in data1 will be normalized using $L^2$ norm. +data1 = labels.zip(normalizer1.transform(features)) + +# Each sample in data2 will be normalized using $L^\infty$ norm. +data2 = labels.zip(normalizer2.transform(features)) +{% endhighlight %} +
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 b478c21537c2a..485abe272326c 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 @@ -31,8 +31,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.{PythonRDD, SerDeUtil} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ -import org.apache.spark.mllib.feature.Word2Vec -import org.apache.spark.mllib.feature.Word2VecModel +import org.apache.spark.mllib.feature._ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.random.{RandomRDDs => RG} @@ -291,6 +290,43 @@ class PythonMLLibAPI extends Serializable { ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha) } + /** + * Java stub for Normalizer.transform() + */ + def normalizeVector(p: Double, vector: Vector): Vector = { + new Normalizer(p).transform(vector) + } + + /** + * Java stub for Normalizer.transform() + */ + def normalizeVector(p: Double, rdd: JavaRDD[Vector]): JavaRDD[Vector] = { + new Normalizer(p).transform(rdd) + } + + /** + * Java stub for IDF.fit(). This stub returns a + * handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on + * exit; see the Py4J documentation. + */ + def fitStandardScaler( + withMean: Boolean, + withStd: Boolean, + data: JavaRDD[Vector]): StandardScalerModel = { + new StandardScaler(withMean, withStd).fit(data.rdd) + } + + /** + * Java stub for IDF.fit(). This stub returns a + * handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on + * exit; see the Py4J documentation. + */ + def fitIDF(minDocFreq: Int, dataset: JavaRDD[Vector]): IDFModel = { + new IDF(minDocFreq).fit(dataset) + } + /** * Java stub for Python mllib Word2Vec fit(). This stub returns a * handle to the Java object instead of the content of the Java object. @@ -328,6 +364,15 @@ class PythonMLLibAPI extends Serializable { model.transform(word) } + /** + * Transforms an RDD of words to its vector representation + * @param rdd an RDD of words + * @return an RDD of vector representations of words + */ + def transform(rdd: JavaRDD[String]): JavaRDD[Vector] = { + rdd.rdd.map(model.transform) + } + def findSynonyms(word: String, num: Int): java.util.List[java.lang.Object] = { val vec = transform(word) findSynonyms(vec, num) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala index 415a845332d45..7358c1c84f79c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.feature import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD @@ -48,4 +49,14 @@ trait VectorTransformer extends Serializable { data.map(x => this.transform(x)) } + /** + * Applies transformation on an JavaRDD[Vector]. + * + * @param data JavaRDD[Vector] to be transformed. + * @return transformed JavaRDD[Vector]. + */ + def transform(data: JavaRDD[Vector]): JavaRDD[Vector] = { + transform(data.rdd) + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index d321994c2a651..f5f7ad613d4c4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -432,7 +432,7 @@ class Word2VecModel private[mllib] ( throw new IllegalStateException(s"$word not in vocabulary") } } - + /** * Find synonyms of a word * @param word a word @@ -443,7 +443,7 @@ class Word2VecModel private[mllib] ( val vector = transform(word) findSynonyms(vector,num) } - + /** * Find synonyms of the vector representation of a word * @param vector vector representation of a word diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index b5a3f22c6907e..324343443ebdb 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -18,59 +18,357 @@ """ Python package for feature in MLlib. """ +import sys +import warnings + +import py4j.protocol +from py4j.protocol import Py4JJavaError +from py4j.java_gateway import JavaObject + +from pyspark import RDD, SparkContext from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.linalg import Vectors, _to_java_object_rdd + +__all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', + 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] + + +# Hack for support float('inf') in Py4j +_old_smart_decode = py4j.protocol.smart_decode + +_float_str_mapping = { + u'nan': u'NaN', + u'inf': u'Infinity', + u'-inf': u'-Infinity', +} + + +def _new_smart_decode(obj): + if isinstance(obj, float): + s = unicode(obj) + return _float_str_mapping.get(s, s) + return _old_smart_decode(obj) + +py4j.protocol.smart_decode = _new_smart_decode + + +# TODO: move these helper functions into utils +_picklable_classes = [ + 'LinkedList', + 'SparseVector', + 'DenseVector', + 'DenseMatrix', + 'Rating', + 'LabeledPoint', +] + + +def _py2java(sc, a): + """ Convert Python object into Java """ + if isinstance(a, RDD): + a = _to_java_object_rdd(a) + elif not isinstance(a, (int, long, float, bool, basestring)): + bytes = bytearray(PickleSerializer().dumps(a)) + a = sc._jvm.SerDe.loads(bytes) + return a + + +def _java2py(sc, r): + if isinstance(r, JavaObject): + clsName = r.getClass().getSimpleName() + if clsName in ("RDD", "JavaRDD"): + if clsName == "RDD": + r = r.toJavaRDD() + jrdd = sc._jvm.SerDe.javaToPython(r) + return RDD(jrdd, sc, AutoBatchedSerializer(PickleSerializer())) -__all__ = ['Word2Vec', 'Word2VecModel'] + elif clsName in _picklable_classes: + r = sc._jvm.SerDe.dumps(r) + if isinstance(r, bytearray): + r = PickleSerializer().loads(str(r)) + return r -class Word2VecModel(object): + +def _callJavaFunc(sc, func, *args): + """ Call Java Function """ - class for Word2Vec model + args = [_py2java(sc, a) for a in args] + return _java2py(sc, func(*args)) + + +def _callAPI(sc, name, *args): + """ Call API in PythonMLLibAPI """ - def __init__(self, sc, java_model): + api = getattr(sc._jvm.PythonMLLibAPI(), name) + return _callJavaFunc(sc, api, *args) + + +class VectorTransformer(object): + """ + :: DeveloperApi :: + + Base class for transformation of a vector or RDD of vector + """ + def transform(self, vector): + """ + Applies transformation on a vector. + + :param vector: vector to be transformed. + """ + raise NotImplementedError + + +class Normalizer(VectorTransformer): + """ + :: Experimental :: + + Normalizes samples individually to unit L\ :sup:`p`\ norm + + For any 1 <= `p` <= float('inf'), normalizes samples using + sum(abs(vector). :sup:`p`) :sup:`(1/p)` as norm. + + For `p` = float('inf'), max(abs(vector)) will be used as norm for normalization. + + >>> v = Vectors.dense(range(3)) + >>> nor = Normalizer(1) + >>> nor.transform(v) + DenseVector([0.0, 0.3333, 0.6667]) + + >>> rdd = sc.parallelize([v]) + >>> nor.transform(rdd).collect() + [DenseVector([0.0, 0.3333, 0.6667])] + + >>> nor2 = Normalizer(float("inf")) + >>> nor2.transform(v) + DenseVector([0.0, 0.5, 1.0]) + """ + def __init__(self, p=2.0): """ - :param sc: Spark context - :param java_model: Handle to Java model object + :param p: Normalization in L^p^ space, p = 2 by default. """ + assert p >= 1.0, "p should be greater than 1.0" + self.p = float(p) + + def transform(self, vector): + """ + Applies unit length normalization on a vector. + + :param vector: vector to be normalized. + :return: normalized vector. If the norm of the input is zero, it + will return the input vector. + """ + sc = SparkContext._active_spark_context + assert sc is not None, "SparkContext should be initialized first" + return _callAPI(sc, "normalizeVector", self.p, vector) + + +class JavaModelWrapper(VectorTransformer): + """ + Wrapper for the model in JVM + """ + def __init__(self, sc, java_model): self._sc = sc self._java_model = java_model def __del__(self): self._sc._gateway.detach(self._java_model) - def transform(self, word): + def transform(self, dataset): + return _callJavaFunc(self._sc, self._java_model.transform, dataset) + + +class StandardScalerModel(JavaModelWrapper): + """ + :: Experimental :: + + Represents a StandardScaler model that can transform vectors. + """ + def transform(self, vector): """ - :param word: a word - :return: vector representation of word + Applies standardization transformation on a vector. + + :param vector: Vector to be standardized. + :return: Standardized vector. If the variance of a column is zero, + it will return default `0.0` for the column with zero variance. + """ + return JavaModelWrapper.transform(self, vector) + + +class StandardScaler(object): + """ + :: Experimental :: + + Standardizes features by removing the mean and scaling to unit + variance using column summary statistics on the samples in the + training set. + >>> vs = [Vectors.dense([-2.0, 2.3, 0]), Vectors.dense([3.8, 0.0, 1.9])] + >>> dataset = sc.parallelize(vs) + >>> standardizer = StandardScaler(True, True) + >>> model = standardizer.fit(dataset) + >>> result = model.transform(dataset) + >>> for r in result.collect(): r + DenseVector([-0.7071, 0.7071, -0.7071]) + DenseVector([0.7071, -0.7071, 0.7071]) + """ + def __init__(self, withMean=False, withStd=True): + """ + :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. + """ + if not (withMean or withStd): + warnings.warn("Both withMean and withStd are false. The model does nothing.") + self.withMean = withMean + self.withStd = withStd + + def fit(self, dataset): + """ + Computes the mean and variance and stores as a model to be used for later scaling. + + :param data: The data used to compute the mean and variance to build + the transformation model. + :return: a StandardScalarModel + """ + sc = dataset.context + jmodel = _callAPI(sc, "fitStandardScaler", self.withMean, self.withStd, dataset) + return StandardScalerModel(sc, jmodel) + + +class HashingTF(object): + """ + :: Experimental :: + + Maps a sequence of terms to their term frequencies using the hashing trick. + + Note: the terms must be hashable (can not be dict/set/list...). + + >>> htf = HashingTF(100) + >>> doc = "a a b b c d".split(" ") + >>> htf.transform(doc) + SparseVector(100, {1: 1.0, 14: 1.0, 31: 2.0, 44: 2.0}) + """ + def __init__(self, numFeatures=1 << 20): + """ + :param numFeatures: number of features (default: 2^20) + """ + self.numFeatures = numFeatures + + def indexOf(self, term): + """ Returns the index of the input term. """ + return hash(term) % self.numFeatures + + def transform(self, document): + """ + Transforms the input document (list of terms) to term frequency vectors, + or transform the RDD of document to RDD of term frequency vectors. + """ + if isinstance(document, RDD): + return document.map(self.transform) + + freq = {} + for term in document: + i = self.indexOf(term) + freq[i] = freq.get(i, 0) + 1.0 + return Vectors.sparse(self.numFeatures, freq.items()) + + +class IDFModel(JavaModelWrapper): + """ + Represents an IDF model that can transform term frequency vectors. + """ + def transform(self, dataset): + """ + Transforms term frequency (TF) vectors to TF-IDF vectors. + + If `minDocFreq` was set for the IDF calculation, + the terms which occur in fewer than `minDocFreq` + documents will have an entry of 0. + + :param dataset: an RDD of term frequency vectors + :return: an RDD of TF-IDF vectors + """ + return JavaModelWrapper.transform(self, dataset) + + +class IDF(object): + """ + :: Experimental :: + + Inverse document frequency (IDF). + + The standard formulation is used: `idf = log((m + 1) / (d(t) + 1))`, + where `m` is the total number of documents and `d(t)` is the number + of documents that contain term `t`. + + This implementation supports filtering out terms which do not appear + in a minimum number of documents (controlled by the variable `minDocFreq`). + For terms that are not in at least `minDocFreq` documents, the IDF is + found as 0, resulting in TF-IDFs of 0. + + >>> n = 4 + >>> freqs = [Vectors.sparse(n, (1, 3), (1.0, 2.0)), + ... Vectors.dense([0.0, 1.0, 2.0, 3.0]), + ... Vectors.sparse(n, [1], [1.0])] + >>> data = sc.parallelize(freqs) + >>> idf = IDF() + >>> model = idf.fit(data) + >>> tfidf = model.transform(data) + >>> for r in tfidf.collect(): r + SparseVector(4, {1: 0.0, 3: 0.5754}) + DenseVector([0.0, 0.0, 1.3863, 0.863]) + SparseVector(4, {1: 0.0}) + """ + def __init__(self, minDocFreq=0): + """ + :param minDocFreq: minimum of documents in which a term + should appear for filtering + """ + self.minDocFreq = minDocFreq + + def fit(self, dataset): + """ + Computes the inverse document frequency. + + :param dataset: an RDD of term frequency vectors + """ + sc = dataset.context + jmodel = _callAPI(sc, "fitIDF", self.minDocFreq, dataset) + return IDFModel(sc, jmodel) + + +class Word2VecModel(JavaModelWrapper): + """ + class for Word2Vec model + """ + def transform(self, word): + """ Transforms a word to its vector representation Note: local use only + + :param word: a word + :return: vector representation of word(s) """ - # TODO: make transform usable in RDD operations from python side - result = self._java_model.transform(word) - return PickleSerializer().loads(str(self._sc._jvm.SerDe.dumps(result))) + try: + return _callJavaFunc(self._sc, self._java_model.transform, word) + except Py4JJavaError: + raise ValueError("%s not found" % word) - def findSynonyms(self, x, num): + def findSynonyms(self, word, num): """ - :param x: a word or a vector representation of word + Find synonyms of a word + + :param word: a word or a vector representation of word :param num: number of synonyms to find :return: array of (word, cosineSimilarity) - Find synonyms of a word - Note: local use only """ - # TODO: make findSynonyms usable in RDD operations from python side - ser = PickleSerializer() - if type(x) == str: - jlist = self._java_model.findSynonyms(x, num) - else: - bytes = bytearray(ser.dumps(_convert_to_vector(x))) - vec = self._sc._jvm.SerDe.loads(bytes) - jlist = self._java_model.findSynonyms(vec, num) - words, similarity = ser.loads(str(self._sc._jvm.SerDe.dumps(jlist))) + words, similarity = _callJavaFunc(self._sc, self._java_model.findSynonyms, word, num) return zip(words, similarity) @@ -85,6 +383,7 @@ class Word2Vec(object): We used skip-gram model in our implementation and hierarchical softmax method to train the model. The variable names in the implementation matches the original C implementation. + For original C implementation, see https://code.google.com/p/word2vec/ For research papers, see Efficient Estimation of Word Representations in Vector Space @@ -95,33 +394,26 @@ class Word2Vec(object): >>> localDoc = [sentence, sentence] >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) >>> model = Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + >>> syms = model.findSynonyms("a", 2) - >>> str(syms[0][0]) - 'b' - >>> str(syms[1][0]) - 'c' - >>> len(syms) - 2 + >>> [s[0] for s in syms] + [u'b', u'c'] >>> vec = model.transform("a") - >>> len(vec) - 10 >>> syms = model.findSynonyms(vec, 2) - >>> str(syms[0][0]) - 'b' - >>> str(syms[1][0]) - 'c' - >>> len(syms) - 2 + >>> [s[0] for s in syms] + [u'b', u'c'] """ def __init__(self): """ Construct Word2Vec instance """ + import random # this can't be on the top because of mllib.random + self.vectorSize = 100 self.learningRate = 0.025 self.numPartitions = 1 self.numIterations = 1 - self.seed = 42L + self.seed = random.randint(0, sys.maxint) def setVectorSize(self, vectorSize): """ @@ -164,20 +456,13 @@ def fit(self, data): Computes the vector representation of each word in vocabulary. :param data: training data. RDD of subtype of Iterable[String] - :return: python Word2VecModel instance + :return: Word2VecModel instance """ sc = data.context - ser = PickleSerializer() - vectorSize = self.vectorSize - learningRate = self.learningRate - numPartitions = self.numPartitions - numIterations = self.numIterations - seed = self.seed - - model = sc._jvm.PythonMLLibAPI().trainWord2Vec( - _to_java_object_rdd(data), vectorSize, - learningRate, numPartitions, numIterations, seed) - return Word2VecModel(sc, model) + jmodel = _callAPI(sc, "trainWord2Vec", data, int(self.vectorSize), + float(self.learningRate), int(self.numPartitions), + int(self.numIterations), long(self.seed)) + return Word2VecModel(sc, jmodel) def _test(): @@ -191,4 +476,8 @@ def _test(): exit(-1) if __name__ == "__main__": + # remove current path from list of search paths to avoid importing mllib.random + # for C{import random}, which is done in an external dependency of pyspark during doctests. + import sys + sys.path.pop(0) _test() diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 773d8d393805d..1b9bf596242df 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -111,6 +111,13 @@ def _vector_size(v): raise TypeError("Cannot treat type %s as a vector" % type(v)) +def _format_float(f, digits=4): + s = str(round(f, digits)) + if '.' in s: + s = s[:s.index('.') + 1 + digits] + return s + + class Vector(object): """ Abstract class for DenseVector and SparseVector @@ -228,7 +235,7 @@ def __str__(self): return "[" + ",".join([str(v) for v in self.array]) + "]" def __repr__(self): - return "DenseVector(%r)" % self.array + return "DenseVector([%s])" % (', '.join(_format_float(i) for i in self.array)) def __eq__(self, other): return isinstance(other, DenseVector) and self.array == other.array @@ -416,7 +423,7 @@ def toArray(self): Returns a copy of this SparseVector as a 1-dimensional NumPy array. """ arr = np.zeros((self.size,), dtype=np.float64) - for i in xrange(self.indices.size): + for i in xrange(len(self.indices)): arr[self.indices[i]] = self.values[i] return arr @@ -431,7 +438,8 @@ def __str__(self): def __repr__(self): inds = self.indices vals = self.values - entries = ", ".join(["{0}: {1}".format(inds[i], vals[i]) for i in xrange(len(inds))]) + entries = ", ".join(["{0}: {1}".format(inds[i], _format_float(vals[i])) + for i in xrange(len(inds))]) return "SparseVector({0}, {{{1}}})".format(self.size, entries) def __eq__(self, other): @@ -491,7 +499,7 @@ def dense(elements): returns a NumPy array. >>> Vectors.dense([1, 2, 3]) - DenseVector(array('d', [1.0, 2.0, 3.0])) + DenseVector([1.0, 2.0, 3.0]) """ return DenseVector(elements) From 47346cd029abc50c70582a721810a7cceb682d8a Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 28 Oct 2014 08:46:31 -0500 Subject: [PATCH 056/115] [SPARK-4116][YARN]Delete the abandoned log4j-spark-container.properties Since its name reduced at https://github.com/apache/spark/pull/560, the log4j-spark-container.properties was never used again. And I have searched its name globally in code and found no cite. Author: WangTaoTheTonic Closes #2977 from WangTaoTheTonic/delLog4j and squashes the following commits: fb2729f [WangTaoTheTonic] delete the log4j file obsoleted --- .../log4j-spark-container.properties | 24 ------------------- 1 file changed, 24 deletions(-) delete mode 100644 yarn/common/src/main/resources/log4j-spark-container.properties diff --git a/yarn/common/src/main/resources/log4j-spark-container.properties b/yarn/common/src/main/resources/log4j-spark-container.properties deleted file mode 100644 index a1e37a0be27dd..0000000000000 --- a/yarn/common/src/main/resources/log4j-spark-container.properties +++ /dev/null @@ -1,24 +0,0 @@ -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. See accompanying LICENSE file. - -# Set everything to be logged to the console -log4j.rootCategory=INFO, console -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - -# Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO -log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO From e8813be6539aba1cd1f8854c204b7938464403ed Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 28 Oct 2014 08:53:10 -0500 Subject: [PATCH 057/115] [SPARK-4095][YARN][Minor]extract val isLaunchingDriver in ClientBase Instead of checking if `args.userClass` is null repeatedly, we extract it to an global val as in `ApplicationMaster`. Author: WangTaoTheTonic Closes #2954 from WangTaoTheTonic/MemUnit and squashes the following commits: 13bda20 [WangTaoTheTonic] extract val isLaunchingDriver in ClientBase --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index fb0e34bf5985e..0417cdd00a393 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -55,6 +55,7 @@ private[spark] trait ClientBase extends Logging { protected val amMemoryOverhead = args.amMemoryOverhead // MB protected val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() + private val isLaunchingDriver = args.userClass != null /** * Fail fast if we have requested more resources per container than is available in the cluster. @@ -267,7 +268,6 @@ private[spark] trait ClientBase extends Logging { // Note that to warn the user about the deprecation in cluster mode, some code from // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition // described above). - val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = @@ -344,7 +344,6 @@ private[spark] trait ClientBase extends Logging { } // Include driver-specific java options if we are launching a driver - val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) @@ -357,7 +356,7 @@ private[spark] trait ClientBase extends Logging { javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) val userClass = - if (args.userClass != null) { + if (isLaunchingDriver) { Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass)) } else { Nil From 0ac52e30552530b247e37a470b8503346f19605c Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 28 Oct 2014 09:51:44 -0500 Subject: [PATCH 058/115] [SPARK-4098][YARN]use appUIAddress instead of appUIHostPort in yarn-client mode https://issues.apache.org/jira/browse/SPARK-4098 Author: WangTaoTheTonic Closes #2958 from WangTaoTheTonic/useAddress and squashes the following commits: 29236e6 [WangTaoTheTonic] use appUIAddress instead of appUIHostPort in yarn-cluster mode --- .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index d948a2aeedd45..59b2b47aed2fe 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -48,7 +48,7 @@ private[spark] class YarnClientSchedulerBackend( val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort - sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIHostPort) } + sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIAddress) } val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ("--arg", hostport) From 7768a800d4c4c19d76cef1ee40af6900bbac821c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 28 Oct 2014 10:14:16 -0700 Subject: [PATCH 059/115] [SPARK-4031] Make torrent broadcast read blocks on use. This avoids reading torrent broadcast variables when they are referenced in the closure but not used in the closure. This is done by using a `lazy val` to read broadcast blocks cc rxin JoshRosen for review Author: Shivaram Venkataraman Closes #2871 from shivaram/broadcast-read-value and squashes the following commits: 1456d65 [Shivaram Venkataraman] Use getUsedTimeMs and remove readObject d6c5ee9 [Shivaram Venkataraman] Use laxy val to implement readBroadcastBlock 0b34df7 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/apache/spark into broadcast-read-value 9cec507 [Shivaram Venkataraman] Test if broadcast variables are read lazily 768b40b [Shivaram Venkataraman] Merge branch 'master' of https://github.com/apache/spark into broadcast-read-value 8792ed8 [Shivaram Venkataraman] Make torrent broadcast read blocks on use. This avoids reading broadcast variables when they are referenced in the closure but not used by the code. --- .../spark/broadcast/TorrentBroadcast.scala | 43 ++++++++++--------- .../scala/org/apache/spark/util/Utils.scala | 15 +++++++ .../spark/broadcast/BroadcastSuite.scala | 30 ++++++++++++- 3 files changed, 67 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 75e64c1bf401e..94142d33369c7 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -56,11 +56,13 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) extends Broadcast[T](id) with Logging with Serializable { /** - * Value of the broadcast object. On driver, this is set directly by the constructor. - * On executors, this is reconstructed by [[readObject]], which builds this value by reading - * blocks from the driver and/or other executors. + * Value of the broadcast object on executors. This is reconstructed by [[readBroadcastBlock]], + * which builds this value by reading blocks from the driver and/or other executors. + * + * On the driver, if the value is required, it is read lazily from the block manager. */ - @transient private var _value: T = obj + @transient private lazy val _value: T = readBroadcastBlock() + /** The compression codec to use, or None if compression is disabled */ @transient private var compressionCodec: Option[CompressionCodec] = _ /** Size of each block. Default value is 4MB. This value is only read by the broadcaster. */ @@ -79,22 +81,24 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) private val broadcastId = BroadcastBlockId(id) /** Total number of blocks this broadcast variable contains. */ - private val numBlocks: Int = writeBlocks() + private val numBlocks: Int = writeBlocks(obj) - override protected def getValue() = _value + override protected def getValue() = { + _value + } /** * Divide the object into multiple blocks and put those blocks in the block manager. - * + * @param value the object to divide * @return number of blocks this broadcast variable is divided into */ - private def writeBlocks(): Int = { + private def writeBlocks(value: T): Int = { // Store a copy of the broadcast variable in the driver so that tasks run on the driver // do not create a duplicate copy of the broadcast variable's value. - SparkEnv.get.blockManager.putSingle(broadcastId, _value, StorageLevel.MEMORY_AND_DISK, + SparkEnv.get.blockManager.putSingle(broadcastId, value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) val blocks = - TorrentBroadcast.blockifyObject(_value, blockSize, SparkEnv.get.serializer, compressionCodec) + TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec) blocks.zipWithIndex.foreach { case (block, i) => SparkEnv.get.blockManager.putBytes( BroadcastBlockId(id, "piece" + i), @@ -157,31 +161,30 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) out.defaultWriteObject() } - /** Used by the JVM when deserializing this object. */ - private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { - in.defaultReadObject() + private def readBroadcastBlock(): T = Utils.tryOrIOException { TorrentBroadcast.synchronized { setConf(SparkEnv.get.conf) SparkEnv.get.blockManager.getLocal(broadcastId).map(_.data.next()) match { case Some(x) => - _value = x.asInstanceOf[T] + x.asInstanceOf[T] case None => logInfo("Started reading broadcast variable " + id) - val start = System.nanoTime() + val startTimeMs = System.currentTimeMillis() val blocks = readBlocks() - val time = (System.nanoTime() - start) / 1e9 - logInfo("Reading broadcast variable " + id + " took " + time + " s") + logInfo("Reading broadcast variable " + id + " took" + Utils.getUsedTimeMs(startTimeMs)) - _value = - TorrentBroadcast.unBlockifyObject[T](blocks, SparkEnv.get.serializer, compressionCodec) + val obj = TorrentBroadcast.unBlockifyObject[T]( + blocks, SparkEnv.get.serializer, compressionCodec) // Store the merged copy in BlockManager so other tasks on this executor don't // need to re-fetch it. SparkEnv.get.blockManager.putSingle( - broadcastId, _value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + broadcastId, obj, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + obj } } } + } 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 4660030155854..612eca308bf0b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -988,6 +988,21 @@ private[spark] object Utils extends Logging { } } + /** + * Execute a block of code that returns a value, re-throwing any non-fatal uncaught + * exceptions as IOException. This is used when implementing Externalizable and Serializable's + * read and write methods, since Java's serializer will not report non-IOExceptions properly; + * see SPARK-4080 for more context. + */ + def tryOrIOException[T](block: => T): T = { + try { + block + } catch { + case e: IOException => throw e + case NonFatal(t) => throw new IOException(t) + } + } + /** Default filtering function for finding call sites using `getCallSite`. */ private def coreExclusionFunction(className: String): Boolean = { // A regular expression to match classes of the "core" Spark API that we want to skip when diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 1014fd62d9a75..b0a70f012f1f3 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -21,11 +21,28 @@ import scala.util.Random import org.scalatest.{Assertions, FunSuite} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkEnv} import org.apache.spark.io.SnappyCompressionCodec +import org.apache.spark.rdd.RDD import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ +// Dummy class that creates a broadcast variable but doesn't use it +class DummyBroadcastClass(rdd: RDD[Int]) extends Serializable { + @transient val list = List(1, 2, 3, 4) + val broadcast = rdd.context.broadcast(list) + val bid = broadcast.id + + def doSomething() = { + rdd.map { x => + val bm = SparkEnv.get.blockManager + // Check if broadcast block was fetched + val isFound = bm.getLocal(BroadcastBlockId(bid)).isDefined + (x, isFound) + }.collect().toSet + } +} + class BroadcastSuite extends FunSuite with LocalSparkContext { private val httpConf = broadcastConf("HttpBroadcastFactory") @@ -105,6 +122,17 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { } } + test("Test Lazy Broadcast variables with TorrentBroadcast") { + val numSlaves = 2 + val conf = torrentConf.clone + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", conf) + val rdd = sc.parallelize(1 to numSlaves) + + val results = new DummyBroadcastClass(rdd).doSomething() + + assert(results.toSet === (1 to numSlaves).map(x => (x, false)).toSet) + } + test("Unpersisting HttpBroadcast on executors only in local mode") { testUnpersistHttpBroadcast(distributed = false, removeFromDriver = false) } From 44d8b45a38c8d934628373a3b21084432516ee00 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 28 Oct 2014 12:29:01 -0700 Subject: [PATCH 060/115] [SPARK-4110] Wrong comments about default settings in spark-daemon.sh In spark-daemon.sh, thare are following comments. # SPARK_CONF_DIR Alternate conf dir. Default is ${SPARK_PREFIX}/conf. # SPARK_LOG_DIR Where log files are stored. PWD by default. But, I think the default value for SPARK_CONF_DIR is `${SPARK_HOME}/conf` and for SPARK_LOG_DIR is `${SPARK_HOME}/logs`. Author: Kousuke Saruta Closes #2972 from sarutak/SPARK-4110 and squashes the following commits: 5a171a2 [Kousuke Saruta] Fixed wrong comments --- sbin/spark-daemon.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index cba475e2dd8c8..89608bc41b71d 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -21,8 +21,8 @@ # # Environment Variables # -# SPARK_CONF_DIR Alternate conf dir. Default is ${SPARK_PREFIX}/conf. -# SPARK_LOG_DIR Where log files are stored. PWD by default. +# SPARK_CONF_DIR Alternate conf dir. Default is ${SPARK_HOME}/conf. +# SPARK_LOG_DIR Where log files are stored. ${SPARK_HOME}/logs by default. # SPARK_MASTER host:path where spark code should be rsync'd from # SPARK_PID_DIR The pid files are stored. /tmp by default. # SPARK_IDENT_STRING A string representing this instance of spark. $USER by default From 1ea3e3dc9dd942402731751089bab2fb6ae29c7b Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 28 Oct 2014 12:31:42 -0700 Subject: [PATCH 061/115] [SPARK-4096][YARN]let ApplicationMaster accept executor memory argument in same format as JVM memory strings Here `ApplicationMaster` accept executor memory argument only in number format, we should let it accept JVM style memory strings as well. Author: WangTaoTheTonic Closes #2955 from WangTaoTheTonic/modifyDesc and squashes the following commits: ab98c70 [WangTaoTheTonic] append parameter passed in 3779767 [WangTaoTheTonic] Update executor memory description in the help message --- .../apache/spark/deploy/yarn/ApplicationMasterArguments.scala | 4 ++-- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 3e6b96fb63cea..5c54e3400301a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import org.apache.spark.util.IntParam +import org.apache.spark.util.{MemoryParam, IntParam} import collection.mutable.ArrayBuffer class ApplicationMasterArguments(val args: Array[String]) { @@ -55,7 +55,7 @@ class ApplicationMasterArguments(val args: Array[String]) { numExecutors = value args = tail - case ("--worker-memory" | "--executor-memory") :: IntParam(value) :: tail => + case ("--worker-memory" | "--executor-memory") :: MemoryParam(value) :: tail => executorMemory = value args = tail diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 0417cdd00a393..8ea0e7cf40a14 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -379,7 +379,7 @@ private[spark] trait ClientBase extends Logging { val amArgs = Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ Seq( - "--executor-memory", args.executorMemory.toString, + "--executor-memory", args.executorMemory.toString + "m", "--executor-cores", args.executorCores.toString, "--num-executors ", args.numExecutors.toString) From 247c529b35d73c5a690bf6512b52999daba1a938 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 28 Oct 2014 12:37:09 -0700 Subject: [PATCH 062/115] [SPARK-3657] yarn alpha YarnRMClientImpl throws NPE appMasterRequest.setTrackingUrl starting spark-shell tgravescs reported this issue. Following is quoted from tgravescs' report. YarnRMClientImpl.registerApplicationMaster can throw null pointer exception when setting the trackingurl if its empty: appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) I hit this just start spark-shell without the tracking url set. 14/09/23 16:18:34 INFO yarn.YarnRMClientImpl: Connecting to ResourceManager at kryptonitered-jt1.red.ygrid.yahoo.com/98.139.154.99:8030 Exception in thread "main" java.lang.NullPointerException at org.apache.hadoop.yarn.proto.YarnServiceProtos$RegisterApplicationMasterRequestProto$Builder.setTrackingUrl(YarnServiceProtos.java:710) at org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl.setTrackingUrl(RegisterApplicationMasterRequestPBImpl.java:132) at org.apache.spark.deploy.yarn.YarnRMClientImpl.registerApplicationMaster(YarnRMClientImpl.scala:102) at org.apache.spark.deploy.yarn.YarnRMClientImpl.register(YarnRMClientImpl.scala:55) at org.apache.spark.deploy.yarn.YarnRMClientImpl.register(YarnRMClientImpl.scala:38) at org.apache.spark.deploy.yarn.ApplicationMaster.registerAM(ApplicationMaster.scala:168) at org.apache.spark.deploy.yarn.ApplicationMaster.runExecutorLauncher(ApplicationMaster.scala:206) at org.apache.spark.deploy.yarn.ApplicationMaster.run(ApplicationMaster.scala:120) Author: Kousuke Saruta Closes #2981 from sarutak/SPARK-3657-2 and squashes the following commits: e2fd6bc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3657 70b8882 [Kousuke Saruta] Fixed NPE thrown --- .../org/apache/spark/deploy/yarn/YarnRMClientImpl.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index 7faf55bc63372..e342cc82f454e 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import scala.collection.{Map, Set} -import java.net.URI; +import java.net.URI import org.apache.hadoop.net.NetUtils import org.apache.hadoop.yarn.api._ @@ -109,7 +109,9 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) // remove the scheme from the url if it exists since Hadoop does not expect scheme - appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) + val uri = new URI(uiAddress) + val authority = if (uri.getScheme == null) uiAddress else uri.getAuthority + appMasterRequest.setTrackingUrl(authority) resourceManager.registerApplicationMaster(appMasterRequest) } From 4d52cec21d98ee16066dcab0cc7959ccf4f412f3 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 28 Oct 2014 12:44:12 -0700 Subject: [PATCH 063/115] [SPARK-4089][Doc][Minor] The version number of Spark in _config.yaml is wrong. The version number of Spark in docs/_config.yaml for master branch should be 1.2.0 for now. Author: Kousuke Saruta Closes #2943 from sarutak/SPARK-4089 and squashes the following commits: aba7fb4 [Kousuke Saruta] Fixed the version number of Spark in _config.yaml --- docs/_config.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index f4bf242ac191b..cdea02fcffbc5 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -11,10 +11,10 @@ kramdown: include: - _static -# These allow the documentation to be updated with nerw releases +# These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 1.0.0-SNAPSHOT -SPARK_VERSION_SHORT: 1.0.0 +SPARK_VERSION: 1.2.0-SNAPSHOT +SPARK_VERSION_SHORT: 1.2.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" MESOS_VERSION: 0.18.1 From 2f254dacf4b7ab9c59c7cef59fd364ca682162ae Mon Sep 17 00:00:00 2001 From: Michael Griffiths Date: Tue, 28 Oct 2014 12:47:21 -0700 Subject: [PATCH 064/115] [SPARK-4065] Add check for IPython on Windows This issue employs logic similar to the bash launcher (pyspark) to check if IPTYHON=1, and if so launch ipython with options in IPYTHON_OPTS. This fix assumes that ipython is available in the system Path, and can be invoked with a plain "ipython" command. Author: Michael Griffiths Closes #2910 from msjgriffiths/pyspark-windows and squashes the following commits: ef34678 [Michael Griffiths] Change build message to comply with [SPARK-3775] 361e3d8 [Michael Griffiths] [SPARK-4065] Add check for IPython on Windows 9ce72d1 [Michael Griffiths] [SPARK-4065] Add check for IPython on Windows --- bin/pyspark2.cmd | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index a0e66abcc26c9..59415e9bdec2c 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -59,7 +59,12 @@ for /f %%i in ('echo %1^| findstr /R "\.py"') do ( ) if [%PYTHON_FILE%] == [] ( - %PYSPARK_PYTHON% + set PYSPARK_SHELL=1 + if [%IPYTHON%] == [1] ( + ipython %IPYTHON_OPTS% + ) else ( + %PYSPARK_PYTHON% + ) ) else ( echo. echo WARNING: Running python applications through ./bin/pyspark.cmd is deprecated as of Spark 1.0. From 6c1b981c3fad671bff4795f061bd40e111956621 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 28 Oct 2014 12:58:25 -0700 Subject: [PATCH 065/115] [SPARK-4058] [PySpark] Log file name is hard coded even though there is a variable '$LOG_FILE ' In a script 'python/run-tests', log file name is represented by a variable 'LOG_FILE' and it is used in run-tests. But, there are some hard-coded log file name in the script. Author: Kousuke Saruta Closes #2905 from sarutak/SPARK-4058 and squashes the following commits: 7710490 [Kousuke Saruta] Fixed python/run-tests not to use hard-coded log file name --- python/run-tests | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/run-tests b/python/run-tests index 80acd002ab7eb..a4f0cac059ff3 100755 --- a/python/run-tests +++ b/python/run-tests @@ -41,7 +41,7 @@ function run_test() { # Fail and exit on the first test failure. if [[ $FAILED != 0 ]]; then - cat unit-tests.log | grep -v "^[0-9][0-9]*" # filter all lines starting with a number. + cat $LOG_FILE | grep -v "^[0-9][0-9]*" # filter all lines starting with a number. echo -en "\033[31m" # Red echo "Had test failures; see logs." echo -en "\033[0m" # No color @@ -87,7 +87,7 @@ function run_streaming_tests() { run_test "pyspark/streaming/tests.py" } -echo "Running PySpark tests. Output is in python/unit-tests.log." +echo "Running PySpark tests. Output is in python/$LOG_FILE." export PYSPARK_PYTHON="python" From 5807cb40ae178f0395c71b967f02aee853ef8bc9 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Tue, 28 Oct 2014 13:36:06 -0700 Subject: [PATCH 066/115] [SPARK-3814][SQL] Support for Bitwise AND(&), OR(|) ,XOR(^), NOT(~) in Spark HQL and SQL Currently there is no support of Bitwise & , | in Spark HiveQl and Spark SQL as well. So this PR support the same. I am closing https://github.com/apache/spark/pull/2926 as it has conflicts to merge. And also added support for Bitwise AND(&), OR(|) ,XOR(^), NOT(~) And I handled all review comments in that PR Author: ravipesala Closes #2961 from ravipesala/SPARK-3814-NEW4 and squashes the following commits: a391c7a [ravipesala] Rebase with master --- .../spark/sql/catalyst/SparkSQLParser.scala | 2 +- .../apache/spark/sql/catalyst/SqlParser.scala | 4 + .../spark/sql/catalyst/dsl/package.scala | 4 + .../sql/catalyst/expressions/arithmetic.scala | 89 +++++++++++++++++++ .../ExpressionEvaluationSuite.scala | 32 +++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 16 ++++ .../org/apache/spark/sql/hive/HiveQl.scala | 4 + .../sql/hive/execution/SQLQuerySuite.scala | 24 +++++ 8 files changed, 174 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala index 219322c015bbf..12e8346a6445d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala @@ -61,7 +61,7 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":", "[", "]", "." + ",", ";", "%", "{", "}", ":", "[", "]", ".", "&", "|", "^", "~" ) override lazy val token: Parser[Token] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 4e967713ede64..0acf7252ba3f0 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -260,6 +260,9 @@ class SqlParser extends AbstractSparkSQLParser { ( "*" ^^^ { (e1: Expression, e2: Expression) => Multiply(e1, e2) } | "/" ^^^ { (e1: Expression, e2: Expression) => Divide(e1, e2) } | "%" ^^^ { (e1: Expression, e2: Expression) => Remainder(e1, e2) } + | "&" ^^^ { (e1: Expression, e2: Expression) => BitwiseAnd(e1, e2) } + | "|" ^^^ { (e1: Expression, e2: Expression) => BitwiseOr(e1, e2) } + | "^" ^^^ { (e1: Expression, e2: Expression) => BitwiseXor(e1, e2) } ) protected lazy val function: Parser[Expression] = @@ -370,6 +373,7 @@ class SqlParser extends AbstractSparkSQLParser { | dotExpressionHeader | ident ^^ UnresolvedAttribute | signedPrimary + | "~" ~> expression ^^ BitwiseNot ) protected lazy val dotExpressionHeader: Parser[Expression] = 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 75b6e37c2a1f9..23cfd483ec410 100755 --- 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 @@ -62,12 +62,16 @@ package object dsl { def unary_- = UnaryMinus(expr) def unary_! = Not(expr) + def unary_~ = BitwiseNot(expr) def + (other: Expression) = Add(expr, other) def - (other: Expression) = Subtract(expr, other) def * (other: Expression) = Multiply(expr, other) def / (other: Expression) = Divide(expr, other) def % (other: Expression) = Remainder(expr, other) + def & (other: Expression) = BitwiseAnd(expr, other) + def | (other: Expression) = BitwiseOr(expr, other) + def ^ (other: Expression) = BitwiseXor(expr, other) def && (other: Expression) = And(expr, other) def || (other: Expression) = Or(expr, other) 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 fe825fdcdae37..83e8466ec2aa7 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 @@ -64,6 +64,23 @@ abstract class BinaryArithmetic extends BinaryExpression { } left.dataType } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + evalInternal(evalE1, evalE2) + } + } + } + + def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = + sys.error(s"BinaryExpressions must either override eval or evalInternal") } case class Add(left: Expression, right: Expression) extends BinaryArithmetic { @@ -100,6 +117,78 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) } +/** + * A function that calculates bitwise and(&) of two numbers. + */ +case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { + def symbol = "&" + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { + case ByteType => (evalE1.asInstanceOf[Byte] & evalE2.asInstanceOf[Byte]).toByte + case ShortType => (evalE1.asInstanceOf[Short] & evalE2.asInstanceOf[Short]).toShort + case IntegerType => evalE1.asInstanceOf[Int] & evalE2.asInstanceOf[Int] + case LongType => evalE1.asInstanceOf[Long] & evalE2.asInstanceOf[Long] + case other => sys.error(s"Unsupported bitwise & operation on ${other}") + } +} + +/** + * A function that calculates bitwise or(|) of two numbers. + */ +case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { + def symbol = "&" + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { + case ByteType => (evalE1.asInstanceOf[Byte] | evalE2.asInstanceOf[Byte]).toByte + case ShortType => (evalE1.asInstanceOf[Short] | evalE2.asInstanceOf[Short]).toShort + case IntegerType => evalE1.asInstanceOf[Int] | evalE2.asInstanceOf[Int] + case LongType => evalE1.asInstanceOf[Long] | evalE2.asInstanceOf[Long] + case other => sys.error(s"Unsupported bitwise | operation on ${other}") + } +} + +/** + * A function that calculates bitwise xor(^) of two numbers. + */ +case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { + def symbol = "^" + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { + case ByteType => (evalE1.asInstanceOf[Byte] ^ evalE2.asInstanceOf[Byte]).toByte + case ShortType => (evalE1.asInstanceOf[Short] ^ evalE2.asInstanceOf[Short]).toShort + case IntegerType => evalE1.asInstanceOf[Int] ^ evalE2.asInstanceOf[Int] + case LongType => evalE1.asInstanceOf[Long] ^ evalE2.asInstanceOf[Long] + case other => sys.error(s"Unsupported bitwise ^ operation on ${other}") + } +} + +/** + * A function that calculates bitwise not(~) of a number. + */ +case class BitwiseNot(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + + def dataType = child.dataType + override def foldable = child.foldable + def nullable = child.nullable + override def toString = s"-$child" + + override def eval(input: Row): Any = { + val evalE = child.eval(input) + if (evalE == null) { + null + } else { + dataType match { + case ByteType => (~(evalE.asInstanceOf[Byte])).toByte + case ShortType => (~(evalE.asInstanceOf[Short])).toShort + case IntegerType => ~(evalE.asInstanceOf[Int]) + case LongType => ~(evalE.asInstanceOf[Long]) + case other => sys.error(s"Unsupported bitwise ~ operation on ${other}") + } + } + } +} + case class MaxOf(left: Expression, right: Expression) extends Expression { type EvaluatedType = Any diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 53c53481f984e..5657bc555edf9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -680,4 +680,36 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Sqrt(Literal(null, DoubleType)), null, new GenericRow(Array[Any](null))) } + + test("Bitwise operations") { + val row = new GenericRow(Array[Any](1, 2, 3, null)) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + + checkEvaluation(BitwiseAnd(c1, c4), null, row) + checkEvaluation(BitwiseAnd(c1, c2), 0, row) + checkEvaluation(BitwiseAnd(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseAnd(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + + checkEvaluation(BitwiseOr(c1, c4), null, row) + checkEvaluation(BitwiseOr(c1, c2), 3, row) + checkEvaluation(BitwiseOr(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseOr(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + + checkEvaluation(BitwiseXor(c1, c4), null, row) + checkEvaluation(BitwiseXor(c1, c2), 3, row) + checkEvaluation(BitwiseXor(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseXor(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + + checkEvaluation(BitwiseNot(c4), null, row) + checkEvaluation(BitwiseNot(c1), -2, row) + checkEvaluation(BitwiseNot(Literal(null, IntegerType)), null, row) + + checkEvaluation(c1 & c2, 0, row) + checkEvaluation(c1 | c2, 3, row) + checkEvaluation(c1 ^ c2, 3, row) + checkEvaluation(~c1, -2, row) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 12e1cfc1cb7eb..1034c2d05f8cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -883,4 +883,20 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { jsonRDD(data).registerTempTable("records") sql("SELECT `key?number1` FROM records") } + + test("SPARK-3814 Support Bitwise & operator") { + checkAnswer(sql("SELECT key&1 FROM testData WHERE key = 1 "), 1) + } + + test("SPARK-3814 Support Bitwise | operator") { + checkAnswer(sql("SELECT key|0 FROM testData WHERE key = 1 "), 1) + } + + test("SPARK-3814 Support Bitwise ^ operator") { + checkAnswer(sql("SELECT key^0 FROM testData WHERE key = 1 "), 1) + } + + test("SPARK-3814 Support Bitwise ~ operator") { + checkAnswer(sql("SELECT ~key FROM testData WHERE key = 1 "), -2) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index aa80b2f04de25..ed07a280398bd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -956,6 +956,7 @@ private[hive] object HiveQl { /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) + case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child)) case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), nodeToExpr(right)) case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) @@ -963,6 +964,9 @@ private[hive] object HiveQl { case Token(DIV(), left :: right:: Nil) => Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType) case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) + case Token("&", left :: right:: Nil) => BitwiseAnd(nodeToExpr(left), nodeToExpr(right)) + case Token("|", left :: right:: Nil) => BitwiseOr(nodeToExpr(left), nodeToExpr(right)) + case Token("^", left :: right:: Nil) => BitwiseXor(nodeToExpr(left), nodeToExpr(right)) case Token("TOK_FUNCTION", Token(SQRT(), Nil) :: arg :: Nil) => Sqrt(nodeToExpr(arg)) /* Comparisons */ 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 fbe6ac765c009..a4aea31d3ff9e 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 @@ -75,4 +75,28 @@ class SQLQuerySuite extends QueryTest { sql("SELECT a.key FROM (SELECT key FROM src) `a`"), sql("SELECT `key` FROM src").collect().toSeq) } + + test("SPARK-3814 Support Bitwise & operator") { + checkAnswer( + sql("SELECT case when 1&1=1 then 1 else 0 end FROM src"), + sql("SELECT 1 FROM src").collect().toSeq) + } + + test("SPARK-3814 Support Bitwise | operator") { + checkAnswer( + sql("SELECT case when 1|0=1 then 1 else 0 end FROM src"), + sql("SELECT 1 FROM src").collect().toSeq) + } + + test("SPARK-3814 Support Bitwise ^ operator") { + checkAnswer( + sql("SELECT case when 1^0=1 then 1 else 0 end FROM src"), + sql("SELECT 1 FROM src").collect().toSeq) + } + + test("SPARK-3814 Support Bitwise ~ operator") { + checkAnswer( + sql("SELECT case when ~1=-2 then 1 else 0 end FROM src"), + sql("SELECT 1 FROM src").collect().toSeq) + } } From 47a40f60d62ea69b659959994918d4c640f39d5b Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 28 Oct 2014 13:43:25 -0700 Subject: [PATCH 067/115] [SPARK-3988][SQL] add public API for date type Add json and python api for date type. By using Pickle, `java.sql.Date` was serialized as calendar, and recognized in python as `datetime.datetime`. Author: Daoyuan Wang Closes #2901 from adrian-wang/spark3988 and squashes the following commits: c51a24d [Daoyuan Wang] convert datetime to date 5670626 [Daoyuan Wang] minor line combine f760d8e [Daoyuan Wang] fix indent 444f100 [Daoyuan Wang] fix a typo 1d74448 [Daoyuan Wang] fix scala style 8d7dd22 [Daoyuan Wang] add json and python api for date type --- python/pyspark/sql.py | 57 +++++++++++++------ .../spark/sql/catalyst/ScalaReflection.scala | 1 + .../spark/sql/catalyst/types/dataTypes.scala | 4 +- .../sql/catalyst/ScalaReflectionSuite.scala | 9 ++- .../org/apache/spark/sql/SQLContext.scala | 10 +++- .../org/apache/spark/sql/json/JsonRDD.scala | 20 +++++-- .../spark/sql/api/java/JavaRowSuite.java | 11 +++- .../java/JavaSideDataTypeConversionSuite.java | 1 + .../ScalaSideDataTypeConversionSuite.scala | 1 + .../org/apache/spark/sql/json/JsonSuite.scala | 9 ++- 10 files changed, 87 insertions(+), 36 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 7daf306f68479..93fd9d49096b8 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -49,7 +49,7 @@ __all__ = [ - "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", + "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", "SQLContext", "HiveContext", "SchemaRDD", "Row"] @@ -132,6 +132,14 @@ class BooleanType(PrimitiveType): """ +class DateType(PrimitiveType): + + """Spark SQL DateType + + The data type representing datetime.date values. + """ + + class TimestampType(PrimitiveType): """Spark SQL TimestampType @@ -438,7 +446,7 @@ def _parse_datatype_json_value(json_value): return _all_complex_types[json_value["type"]].fromJson(json_value) -# Mapping Python types to Spark SQL DateType +# Mapping Python types to Spark SQL DataType _type_mappings = { bool: BooleanType, int: IntegerType, @@ -448,8 +456,8 @@ def _parse_datatype_json_value(json_value): unicode: StringType, bytearray: BinaryType, decimal.Decimal: DecimalType, + datetime.date: DateType, datetime.datetime: TimestampType, - datetime.date: TimestampType, datetime.time: TimestampType, } @@ -656,10 +664,10 @@ def _infer_schema_type(obj, dataType): """ Fill the dataType with types infered from obj - >>> schema = _parse_schema_abstract("a b c") - >>> row = (1, 1.0, "str") + >>> schema = _parse_schema_abstract("a b c d") + >>> row = (1, 1.0, "str", datetime.date(2014, 10, 10)) >>> _infer_schema_type(row, schema) - StructType...IntegerType...DoubleType...StringType... + StructType...IntegerType...DoubleType...StringType...DateType... >>> row = [[1], {"key": (1, 2.0)}] >>> schema = _parse_schema_abstract("a[] b{c d}") >>> _infer_schema_type(row, schema) @@ -703,6 +711,7 @@ def _infer_schema_type(obj, dataType): DecimalType: (decimal.Decimal,), StringType: (str, unicode), BinaryType: (bytearray,), + DateType: (datetime.date,), TimestampType: (datetime.datetime,), ArrayType: (list, tuple, array), MapType: (dict,), @@ -740,7 +749,7 @@ def _verify_type(obj, dataType): # subclass of them can not be deserialized in JVM if type(obj) not in _acceptable_types[_type]: - raise TypeError("%s can not accept abject in type %s" + raise TypeError("%s can not accept object in type %s" % (dataType, type(obj))) if isinstance(dataType, ArrayType): @@ -767,7 +776,7 @@ def _restore_object(dataType, obj): """ Restore object during unpickling. """ # use id(dataType) as key to speed up lookup in dict # Because of batched pickling, dataType will be the - # same object in mose cases. + # same object in most cases. k = id(dataType) cls = _cached_cls.get(k) if cls is None: @@ -782,6 +791,10 @@ def _restore_object(dataType, obj): def _create_object(cls, v): """ Create an customized object with class `cls`. """ + # datetime.date would be deserialized as datetime.datetime + # from java type, so we need to set it back. + if cls is datetime.date and isinstance(v, datetime.datetime): + return v.date() return cls(v) if v is not None else v @@ -795,14 +808,16 @@ def getter(self): return getter -def _has_struct(dt): - """Return whether `dt` is or has StructType in it""" +def _has_struct_or_date(dt): + """Return whether `dt` is or has StructType/DateType in it""" if isinstance(dt, StructType): return True elif isinstance(dt, ArrayType): - return _has_struct(dt.elementType) + return _has_struct_or_date(dt.elementType) elif isinstance(dt, MapType): - return _has_struct(dt.valueType) + return _has_struct_or_date(dt.valueType) + elif isinstance(dt, DateType): + return True return False @@ -815,7 +830,7 @@ def _create_properties(fields): or keyword.iskeyword(name)): warnings.warn("field name %s can not be accessed in Python," "use position to access it instead" % name) - if _has_struct(f.dataType): + if _has_struct_or_date(f.dataType): # delay creating object until accessing it getter = _create_getter(f.dataType, i) else: @@ -870,6 +885,9 @@ def Dict(d): return Dict + elif isinstance(dataType, DateType): + return datetime.date + elif not isinstance(dataType, StructType): raise Exception("unexpected data type: %s" % dataType) @@ -1068,8 +1086,9 @@ def applySchema(self, rdd, schema): >>> srdd2.collect() [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] - >>> from datetime import datetime + >>> from datetime import date, datetime >>> rdd = sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + ... date(2010, 1, 1), ... datetime(2010, 1, 1, 1, 1, 1), ... {"a": 1}, (2,), [1, 2, 3], None)]) >>> schema = StructType([ @@ -1079,6 +1098,7 @@ def applySchema(self, rdd, schema): ... StructField("short2", ShortType(), False), ... StructField("int", IntegerType(), False), ... StructField("float", FloatType(), False), + ... StructField("date", DateType(), False), ... StructField("time", TimestampType(), False), ... StructField("map", ... MapType(StringType(), IntegerType(), False), False), @@ -1088,10 +1108,11 @@ def applySchema(self, rdd, schema): ... StructField("null", DoubleType(), True)]) >>> srdd = sqlCtx.applySchema(rdd, schema) >>> results = srdd.map( - ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.time, - ... x.map["a"], x.struct.b, x.list, x.null)) - >>> results.collect()[0] - (127, -128, -32768, 32767, 2147483647, 1.0, ...(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) + ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.date, + ... x.time, x.map["a"], x.struct.b, x.list, x.null)) + >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE + (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1), + datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) >>> srdd.registerTempTable("table2") >>> sqlCtx.sql( 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 7d930fccd52d1..d76c743d3f652 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 @@ -112,6 +112,7 @@ object ScalaReflection { case obj: FloatType.JvmType => FloatType case obj: DoubleType.JvmType => DoubleType case obj: DecimalType.JvmType => DecimalType + case obj: DateType.JvmType => DateType case obj: TimestampType.JvmType => TimestampType case null => NullType // For other cases, there is no obvious mapping from the type of the given object to a diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 0cf139ebde417..b9cf37d53ffd2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -91,6 +91,7 @@ object DataType { | "BinaryType" ^^^ BinaryType | "BooleanType" ^^^ BooleanType | "DecimalType" ^^^ DecimalType + | "DateType" ^^^ DateType | "TimestampType" ^^^ TimestampType ) @@ -198,7 +199,8 @@ trait PrimitiveType extends DataType { } object PrimitiveType { - private[sql] val all = Seq(DecimalType, TimestampType, BinaryType) ++ NativeType.all + private[sql] val all = Seq(DecimalType, DateType, TimestampType, BinaryType) ++ + NativeType.all private[sql] val nameToType = all.map(t => t.typeName -> t).toMap } 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 488e373854bb3..430f0664b7d58 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst import java.math.BigInteger -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.scalatest.FunSuite @@ -43,6 +43,7 @@ case class NullableData( booleanField: java.lang.Boolean, stringField: String, decimalField: BigDecimal, + dateField: Date, timestampField: Timestamp, binaryField: Array[Byte]) @@ -96,6 +97,7 @@ class ScalaReflectionSuite extends FunSuite { StructField("booleanField", BooleanType, nullable = true), StructField("stringField", StringType, nullable = true), StructField("decimalField", DecimalType, nullable = true), + StructField("dateField", DateType, nullable = true), StructField("timestampField", TimestampType, nullable = true), StructField("binaryField", BinaryType, nullable = true))), nullable = true)) @@ -199,8 +201,11 @@ class ScalaReflectionSuite extends FunSuite { // DecimalType assert(DecimalType === typeOfObject(BigDecimal("1.7976931348623157E318"))) + // DateType + assert(DateType === typeOfObject(Date.valueOf("2014-07-25"))) + // TimestampType - assert(TimestampType === typeOfObject(java.sql.Timestamp.valueOf("2014-07-25 10:26:00"))) + assert(TimestampType === typeOfObject(Timestamp.valueOf("2014-07-25 10:26:00"))) // NullType assert(NullType === typeOfObject(null)) 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 c4f4ef01d78df..ca8706ee68697 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 @@ -444,6 +444,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case ByteType => true case ShortType => true case FloatType => true + case DateType => true case TimestampType => true case ArrayType(_, _) => true case MapType(_, _, _) => true @@ -452,9 +453,9 @@ class SQLContext(@transient val sparkContext: SparkContext) } // Converts value to the type specified by the data type. - // Because Python does not have data types for TimestampType, FloatType, ShortType, and - // ByteType, we need to explicitly convert values in columns of these data types to the desired - // JVM data types. + // Because Python does not have data types for DateType, TimestampType, FloatType, ShortType, + // and ByteType, we need to explicitly convert values in columns of these data types to the + // desired JVM data types. def convert(obj: Any, dataType: DataType): Any = (obj, dataType) match { // TODO: We should check nullable case (null, _) => null @@ -474,6 +475,9 @@ class SQLContext(@transient val sparkContext: SparkContext) case (e, f) => convert(e, f.dataType) }): Row + case (c: java.util.Calendar, DateType) => + new java.sql.Date(c.getTime().getTime()) + case (c: java.util.Calendar, TimestampType) => new java.sql.Timestamp(c.getTime().getTime()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index bf32da1b7181e..047dc85df6c1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.json import scala.collection.Map import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import com.fasterxml.jackson.core.JsonProcessingException import com.fasterxml.jackson.databind.ObjectMapper @@ -372,13 +372,20 @@ private[sql] object JsonRDD extends Logging { } } + private def toDate(value: Any): Date = { + value match { + // only support string as date + case value: java.lang.String => Date.valueOf(value) + } + } + private def toTimestamp(value: Any): Timestamp = { value match { - case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) - case value: java.lang.Long => new Timestamp(value) - case value: java.lang.String => Timestamp.valueOf(value) - } - } + case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) + case value: java.lang.Long => new Timestamp(value) + case value: java.lang.String => Timestamp.valueOf(value) + } + } private[json] def enforceCorrectType(value: Any, desiredType: DataType): Any ={ if (value == null) { @@ -396,6 +403,7 @@ private[sql] object JsonRDD extends Logging { case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], struct) + case DateType => toDate(value) case TimestampType => toTimestamp(value) } } diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java index 52d07b5425cc3..bc5cd66482add 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java @@ -18,6 +18,7 @@ package org.apache.spark.sql.api.java; import java.math.BigDecimal; +import java.sql.Date; import java.sql.Timestamp; import java.util.Arrays; import java.util.HashMap; @@ -39,6 +40,7 @@ public class JavaRowSuite { private boolean booleanValue; private String stringValue; private byte[] binaryValue; + private Date dateValue; private Timestamp timestampValue; @Before @@ -53,6 +55,7 @@ public void setUp() { booleanValue = true; stringValue = "this is a string"; binaryValue = stringValue.getBytes(); + dateValue = Date.valueOf("2014-06-30"); timestampValue = Timestamp.valueOf("2014-06-30 09:20:00.0"); } @@ -76,6 +79,7 @@ public void constructSimpleRow() { new Boolean(booleanValue), stringValue, // StringType binaryValue, // BinaryType + dateValue, // DateType timestampValue, // TimestampType null // null ); @@ -114,9 +118,10 @@ public void constructSimpleRow() { Assert.assertEquals(stringValue, simpleRow.getString(15)); Assert.assertEquals(stringValue, simpleRow.get(15)); Assert.assertEquals(binaryValue, simpleRow.get(16)); - Assert.assertEquals(timestampValue, simpleRow.get(17)); - Assert.assertEquals(true, simpleRow.isNullAt(18)); - Assert.assertEquals(null, simpleRow.get(18)); + Assert.assertEquals(dateValue, simpleRow.get(17)); + Assert.assertEquals(timestampValue, simpleRow.get(18)); + Assert.assertEquals(true, simpleRow.isNullAt(19)); + Assert.assertEquals(null, simpleRow.get(19)); } @Test diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java index d099a48a1f4b6..d04396a5f8ec2 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java @@ -39,6 +39,7 @@ public void createDataTypes() { checkDataType(DataType.StringType); checkDataType(DataType.BinaryType); checkDataType(DataType.BooleanType); + checkDataType(DataType.DateType); checkDataType(DataType.TimestampType); checkDataType(DataType.DecimalType); checkDataType(DataType.DoubleType); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala index ff1debff0f8c1..8415af41be3af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala @@ -38,6 +38,7 @@ class ScalaSideDataTypeConversionSuite extends FunSuite { checkDataType(org.apache.spark.sql.StringType) checkDataType(org.apache.spark.sql.BinaryType) checkDataType(org.apache.spark.sql.BooleanType) + checkDataType(org.apache.spark.sql.DateType) checkDataType(org.apache.spark.sql.TimestampType) checkDataType(org.apache.spark.sql.DecimalType) checkDataType(org.apache.spark.sql.DoubleType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 1ae75546aada1..ce6184f5d8c9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.SQLConf import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ -import java.sql.Timestamp +import java.sql.{Date, Timestamp} class JsonSuite extends QueryTest { import TestJsonData._ @@ -58,8 +58,11 @@ class JsonSuite extends QueryTest { checkTypePromotion(new Timestamp(intNumber), enforceCorrectType(intNumber, TimestampType)) checkTypePromotion(new Timestamp(intNumber.toLong), enforceCorrectType(intNumber.toLong, TimestampType)) - val strDate = "2014-09-30 12:34:56" - checkTypePromotion(Timestamp.valueOf(strDate), enforceCorrectType(strDate, TimestampType)) + val strTime = "2014-09-30 12:34:56" + checkTypePromotion(Timestamp.valueOf(strTime), enforceCorrectType(strTime, TimestampType)) + + val strDate = "2014-10-15" + checkTypePromotion(Date.valueOf(strDate), enforceCorrectType(strDate, DateType)) } test("Get compatible type") { From abcafcfba38d7c8dba68a5510475c5c49ae54d92 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 28 Oct 2014 14:26:57 -0700 Subject: [PATCH 068/115] [Spark 3922] Refactor spark-core to use Utils.UTF_8 A global UTF8 constant is very helpful to handle encoding problems when converting between String and bytes. There are several solutions here: 1. Add `val UTF_8 = Charset.forName("UTF-8")` to Utils.scala 2. java.nio.charset.StandardCharsets.UTF_8 (require JDK7) 3. io.netty.util.CharsetUtil.UTF_8 4. com.google.common.base.Charsets.UTF_8 5. org.apache.commons.lang.CharEncoding.UTF_8 6. org.apache.commons.lang3.CharEncoding.UTF_8 IMO, I prefer option 1) because people can find it easily. This is a PR for option 1) and only fixes Spark Core. Author: zsxwing Closes #2781 from zsxwing/SPARK-3922 and squashes the following commits: f974edd [zsxwing] Merge branch 'master' into SPARK-3922 2d27423 [zsxwing] Refactor spark-core to use Refactor spark-core to use Utils.UTF_8 --- .../scala/org/apache/spark/SparkSaslClient.scala | 7 ++++--- .../scala/org/apache/spark/SparkSaslServer.scala | 10 ++++++---- .../org/apache/spark/api/python/PythonRDD.scala | 9 ++++----- .../python/WriteInputFormatTestDataGenerator.scala | 5 +++-- .../apache/spark/deploy/worker/DriverRunner.scala | 4 ++-- .../apache/spark/deploy/worker/ExecutorRunner.scala | 4 ++-- .../network/netty/client/BlockFetchingClient.scala | 4 ++-- .../netty/client/BlockFetchingClientHandler.scala | 5 +++-- .../spark/network/netty/server/BlockServer.scala | 4 ++-- .../netty/server/BlockServerChannelInitializer.scala | 6 +++--- .../apache/spark/network/nio/ConnectionManager.scala | 4 +++- .../scala/org/apache/spark/network/nio/Message.scala | 4 +++- .../client/BlockFetchingClientHandlerSuite.scala | 3 ++- .../netty/server/BlockHeaderEncoderSuite.scala | 8 ++++---- .../org/apache/spark/util/FileAppenderSuite.scala | 12 ++++++------ .../scala/org/apache/spark/util/UtilsSuite.scala | 12 ++++++------ 16 files changed, 55 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index 65003b6ac6a0a..a954fcc0c31fa 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -17,7 +17,6 @@ package org.apache.spark -import java.io.IOException import javax.security.auth.callback.Callback import javax.security.auth.callback.CallbackHandler import javax.security.auth.callback.NameCallback @@ -31,6 +30,8 @@ import javax.security.sasl.SaslException import scala.collection.JavaConversions.mapAsJavaMap +import com.google.common.base.Charsets.UTF_8 + /** * Implements SASL Client logic for Spark */ @@ -111,10 +112,10 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes(UTF_8)) private val secretKey = securityMgr.getSecretKey() private val userPassword: Array[Char] = SparkSaslServer.encodePassword( - if (secretKey != null) secretKey.getBytes("utf-8") else "".getBytes("utf-8")) + if (secretKey != null) secretKey.getBytes(UTF_8) else "".getBytes(UTF_8)) /** * Implementation used to respond to SASL request from the server. diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index f6b0a9132aca4..7c2afb364661f 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -28,6 +28,8 @@ import javax.security.sasl.Sasl import javax.security.sasl.SaslException import javax.security.sasl.SaslServer import scala.collection.JavaConversions.mapAsJavaMap + +import com.google.common.base.Charsets.UTF_8 import org.apache.commons.net.util.Base64 /** @@ -89,7 +91,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi extends CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes(UTF_8)) override def handle(callbacks: Array[Callback]) { logDebug("In the sasl server callback handler") @@ -101,7 +103,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi case pc: PasswordCallback => { logDebug("handle: SASL server callback: setting userPassword") val password: Array[Char] = - SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes("utf-8")) + SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes(UTF_8)) pc.setPassword(password) } case rc: RealmCallback => { @@ -159,7 +161,7 @@ private[spark] object SparkSaslServer { * @return Base64-encoded string */ def encodeIdentifier(identifier: Array[Byte]): String = { - new String(Base64.encodeBase64(identifier), "utf-8") + new String(Base64.encodeBase64(identifier), UTF_8) } /** @@ -168,7 +170,7 @@ private[spark] object SparkSaslServer { * @return password as a char array. */ def encodePassword(password: Array[Byte]): Array[Char] = { - new String(Base64.encodeBase64(password), "utf-8").toCharArray() + new String(Base64.encodeBase64(password), UTF_8).toCharArray() } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 163dca6cade5a..61b125ef7c6c1 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -19,7 +19,6 @@ package org.apache.spark.api.python import java.io._ import java.net._ -import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ @@ -27,6 +26,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.existentials +import com.google.common.base.Charsets.UTF_8 import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.hadoop.conf.Configuration @@ -134,7 +134,7 @@ private[spark] class PythonRDD( val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj, "utf-8"), + throw new PythonException(new String(obj, UTF_8), writerThread.exception.getOrElse(null)) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still @@ -318,7 +318,6 @@ private object SpecialLengths { } private[spark] object PythonRDD extends Logging { - val UTF8 = Charset.forName("UTF-8") // remember the broadcasts sent to each worker private val workerBroadcasts = new mutable.WeakHashMap[Socket, mutable.Set[Long]]() @@ -586,7 +585,7 @@ private[spark] object PythonRDD extends Logging { } def writeUTF(str: String, dataOut: DataOutputStream) { - val bytes = str.getBytes(UTF8) + val bytes = str.getBytes(UTF_8) dataOut.writeInt(bytes.length) dataOut.write(bytes) } @@ -849,7 +848,7 @@ private[spark] object PythonRDD extends Logging { private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { - override def call(arr: Array[Byte]) : String = new String(arr, PythonRDD.UTF8) + override def call(arr: Array[Byte]) : String = new String(arr, UTF_8) } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index d11db978b842e..e9ca9166eb4d6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -18,7 +18,8 @@ package org.apache.spark.api.python import java.io.{DataOutput, DataInput} -import java.nio.charset.Charset + +import com.google.common.base.Charsets.UTF_8 import org.apache.hadoop.io._ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat @@ -136,7 +137,7 @@ object WriteInputFormatTestDataGenerator { sc.parallelize(intKeys).saveAsSequenceFile(intPath) sc.parallelize(intKeys.map{ case (k, v) => (k.toDouble, v) }).saveAsSequenceFile(doublePath) sc.parallelize(intKeys.map{ case (k, v) => (k.toString, v) }).saveAsSequenceFile(textPath) - sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(Charset.forName("UTF-8"))) } + sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(UTF_8)) } ).saveAsSequenceFile(bytesPath) val bools = Seq((1, true), (2, true), (2, false), (3, true), (2, false), (1, false)) sc.parallelize(bools).saveAsSequenceFile(boolPath) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 9f9911762505a..3bf0b9492df0a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions._ import scala.collection.Map import akka.actor.ActorRef -import com.google.common.base.Charsets +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileUtil, Path} @@ -178,7 +178,7 @@ private[spark] class DriverRunner( val stderr = new File(baseDir, "stderr") val header = "Launch Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) - Files.append(header, stderr, Charsets.UTF_8) + Files.append(header, stderr, UTF_8) CommandUtils.redirectStream(process.getErrorStream, stderr) } runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 71d7385b08eb9..030a651469b64 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker import java.io._ import akka.actor.ActorRef -import com.google.common.base.Charsets +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.spark.{SparkConf, Logging} @@ -151,7 +151,7 @@ private[spark] class ExecutorRunner( stdoutAppender = FileAppender(process.getInputStream, stdout, conf) val stderr = new File(executorDir, "stderr") - Files.write(header, stderr, Charsets.UTF_8) + Files.write(header, stderr, UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) state = ExecutorState.RUNNING diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala index 5aea7ba2f3673..3ab13b96d7f00 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala @@ -19,13 +19,13 @@ package org.apache.spark.network.netty.client import java.util.concurrent.TimeoutException +import com.google.common.base.Charsets.UTF_8 import io.netty.bootstrap.Bootstrap import io.netty.buffer.PooledByteBufAllocator import io.netty.channel.socket.SocketChannel import io.netty.channel.{ChannelFutureListener, ChannelFuture, ChannelInitializer, ChannelOption} import io.netty.handler.codec.LengthFieldBasedFrameDecoder import io.netty.handler.codec.string.StringEncoder -import io.netty.util.CharsetUtil import org.apache.spark.Logging @@ -61,7 +61,7 @@ class BlockFetchingClient(factory: BlockFetchingClientFactory, hostname: String, b.handler(new ChannelInitializer[SocketChannel] { override def initChannel(ch: SocketChannel): Unit = { ch.pipeline - .addLast("encoder", new StringEncoder(CharsetUtil.UTF_8)) + .addLast("encoder", new StringEncoder(UTF_8)) // maxFrameLength = 2G, lengthFieldOffset = 0, lengthFieldLength = 4 .addLast("framedLengthDecoder", new LengthFieldBasedFrameDecoder(Int.MaxValue, 0, 4)) .addLast("handler", handler) diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala index 83265b164299d..d9d3f7bef0b20 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala @@ -17,6 +17,7 @@ package org.apache.spark.network.netty.client +import com.google.common.base.Charsets.UTF_8 import io.netty.buffer.ByteBuf import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} @@ -67,7 +68,7 @@ class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] wi val blockIdLen = in.readInt() val blockIdBytes = new Array[Byte](math.abs(blockIdLen)) in.readBytes(blockIdBytes) - val blockId = new String(blockIdBytes) + val blockId = new String(blockIdBytes, UTF_8) val blockSize = totalLen - math.abs(blockIdLen) - 4 def server = ctx.channel.remoteAddress.toString @@ -76,7 +77,7 @@ class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] wi if (blockIdLen < 0) { val errorMessageBytes = new Array[Byte](blockSize) in.readBytes(errorMessageBytes) - val errorMsg = new String(errorMessageBytes) + val errorMsg = new String(errorMessageBytes, UTF_8) logTrace(s"Received block $blockId ($blockSize B) with error $errorMsg from $server") val listener = outstandingRequests.get(blockId) diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala index 7b2f9a8d4dfd0..9194c7ced3547 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala @@ -19,6 +19,7 @@ package org.apache.spark.network.netty.server import java.net.InetSocketAddress +import com.google.common.base.Charsets.UTF_8 import io.netty.bootstrap.ServerBootstrap import io.netty.buffer.PooledByteBufAllocator import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} @@ -30,7 +31,6 @@ import io.netty.channel.socket.nio.NioServerSocketChannel import io.netty.channel.socket.oio.OioServerSocketChannel import io.netty.handler.codec.LineBasedFrameDecoder import io.netty.handler.codec.string.StringDecoder -import io.netty.util.CharsetUtil import org.apache.spark.{Logging, SparkConf} import org.apache.spark.network.netty.NettyConfig @@ -131,7 +131,7 @@ class BlockServer(conf: NettyConfig, dataProvider: BlockDataProvider) extends Lo override def initChannel(ch: SocketChannel): Unit = { ch.pipeline .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("stringDecoder", new StringDecoder(UTF_8)) .addLast("blockHeaderEncoder", new BlockHeaderEncoder) .addLast("handler", new BlockServerHandler(dataProvider)) } diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala index cc70bd0c5c477..188154d51d0eb 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala @@ -17,13 +17,13 @@ package org.apache.spark.network.netty.server +import com.google.common.base.Charsets.UTF_8 import io.netty.channel.ChannelInitializer import io.netty.channel.socket.SocketChannel import io.netty.handler.codec.LineBasedFrameDecoder import io.netty.handler.codec.string.StringDecoder -import io.netty.util.CharsetUtil -import org.apache.spark.storage.BlockDataProvider +import org.apache.spark.storage.BlockDataProvider /** Channel initializer that sets up the pipeline for the BlockServer. */ private[netty] @@ -33,7 +33,7 @@ class BlockServerChannelInitializer(dataProvider: BlockDataProvider) override def initChannel(ch: SocketChannel): Unit = { ch.pipeline .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("stringDecoder", new StringDecoder(UTF_8)) .addLast("blockHeaderEncoder", new BlockHeaderEncoder) .addLast("handler", new BlockServerHandler(dataProvider)) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index bda4bf50932c3..8408b75bb4d65 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -31,6 +31,8 @@ import scala.concurrent.duration._ import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.language.postfixOps +import com.google.common.base.Charsets.UTF_8 + import org.apache.spark._ import org.apache.spark.util.Utils @@ -923,7 +925,7 @@ private[nio] class ConnectionManager( val errorMsgByteBuf = ackMessage.asInstanceOf[BufferMessage].buffers.head val errorMsgBytes = new Array[Byte](errorMsgByteBuf.limit()) errorMsgByteBuf.get(errorMsgBytes) - val errorMsg = new String(errorMsgBytes, "utf-8") + val errorMsg = new String(errorMsgBytes, UTF_8) val e = new IOException( s"sendMessageReliably failed with ACK that signalled a remote error: $errorMsg") if (!promise.tryFailure(e)) { diff --git a/core/src/main/scala/org/apache/spark/network/nio/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala index 3ad04591da658..fb4a979b824c3 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala @@ -22,6 +22,8 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer +import com.google.common.base.Charsets.UTF_8 + import org.apache.spark.util.Utils private[nio] abstract class Message(val typ: Long, val id: Int) { @@ -92,7 +94,7 @@ private[nio] object Message { */ def createErrorMessage(exception: Exception, ackId: Int): BufferMessage = { val exceptionString = Utils.exceptionString(exception) - val serializedExceptionString = ByteBuffer.wrap(exceptionString.getBytes("utf-8")) + val serializedExceptionString = ByteBuffer.wrap(exceptionString.getBytes(UTF_8)) val errorMessage = createBufferMessage(serializedExceptionString, ackId) errorMessage.hasError = true errorMessage diff --git a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala index 903ab09ae4322..f629322ff6838 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.network.netty.client import java.nio.ByteBuffer +import com.google.common.base.Charsets.UTF_8 import io.netty.buffer.Unpooled import io.netty.channel.embedded.EmbeddedChannel @@ -42,7 +43,7 @@ class BlockFetchingClientHandlerSuite extends FunSuite with PrivateMethodTester parsedBlockId = bid val bytes = new Array[Byte](refCntBuf.byteBuffer().remaining) refCntBuf.byteBuffer().get(bytes) - parsedBlockData = new String(bytes) + parsedBlockData = new String(bytes, UTF_8) } } ) diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala index 3ee281cb1350b..3f8d0cf8f3ff3 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala @@ -17,12 +17,12 @@ package org.apache.spark.network.netty.server +import com.google.common.base.Charsets.UTF_8 import io.netty.buffer.ByteBuf import io.netty.channel.embedded.EmbeddedChannel import org.scalatest.FunSuite - class BlockHeaderEncoderSuite extends FunSuite { test("encode normal block data") { @@ -35,7 +35,7 @@ class BlockHeaderEncoderSuite extends FunSuite { val blockIdBytes = new Array[Byte](blockId.length) out.readBytes(blockIdBytes) - assert(new String(blockIdBytes) === blockId) + assert(new String(blockIdBytes, UTF_8) === blockId) assert(out.readableBytes() === 0) channel.close() @@ -52,11 +52,11 @@ class BlockHeaderEncoderSuite extends FunSuite { val blockIdBytes = new Array[Byte](blockId.length) out.readBytes(blockIdBytes) - assert(new String(blockIdBytes) === blockId) + assert(new String(blockIdBytes, UTF_8) === blockId) val errorMsgBytes = new Array[Byte](errorMsg.length) out.readBytes(errorMsgBytes) - assert(new String(errorMsgBytes) === errorMsg) + assert(new String(errorMsgBytes, UTF_8) === errorMsg) assert(out.readableBytes() === 0) channel.close() diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index d2bee448d4d3b..4dc5b6103db74 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -18,13 +18,13 @@ package org.apache.spark.util import java.io._ -import java.nio.charset.Charset import scala.collection.mutable.HashSet import scala.reflect._ import org.scalatest.{BeforeAndAfter, FunSuite} +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.spark.{Logging, SparkConf} @@ -44,11 +44,11 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { test("basic file appender") { val testString = (1 to 1000).mkString(", ") - val inputStream = new ByteArrayInputStream(testString.getBytes(Charset.forName("UTF-8"))) + val inputStream = new ByteArrayInputStream(testString.getBytes(UTF_8)) val appender = new FileAppender(inputStream, testFile) inputStream.close() appender.awaitTermination() - assert(Files.toString(testFile, Charset.forName("UTF-8")) === testString) + assert(Files.toString(testFile, UTF_8) === testString) } test("rolling file appender - time-based rolling") { @@ -96,7 +96,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { val allGeneratedFiles = new HashSet[String]() val items = (1 to 10).map { _.toString * 10000 } for (i <- 0 until items.size) { - testOutputStream.write(items(i).getBytes(Charset.forName("UTF-8"))) + testOutputStream.write(items(i).getBytes(UTF_8)) testOutputStream.flush() allGeneratedFiles ++= RollingFileAppender.getSortedRolledOverFiles( testFile.getParentFile.toString, testFile.getName).map(_.toString) @@ -199,7 +199,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { // send data to appender through the input stream, and wait for the data to be written val expectedText = textToAppend.mkString("") for (i <- 0 until textToAppend.size) { - outputStream.write(textToAppend(i).getBytes(Charset.forName("UTF-8"))) + outputStream.write(textToAppend(i).getBytes(UTF_8)) outputStream.flush() Thread.sleep(sleepTimeBetweenTexts) } @@ -214,7 +214,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { logInfo("Filtered files: \n" + generatedFiles.mkString("\n")) assert(generatedFiles.size > 1) val allText = generatedFiles.map { file => - Files.toString(file, Charset.forName("UTF-8")) + Files.toString(file, UTF_8) }.mkString("") assert(allText === expectedText) generatedFiles diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index ea7ef0524d1e1..65579bb9affc7 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -23,7 +23,7 @@ import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStr import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} -import com.google.common.base.Charsets +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.scalatest.FunSuite @@ -118,7 +118,7 @@ class UtilsSuite extends FunSuite { tmpDir2.deleteOnExit() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(UTF_8)) f1.close() // Read first few bytes @@ -146,9 +146,9 @@ class UtilsSuite extends FunSuite { val tmpDir = Utils.createTempDir() tmpDir.deleteOnExit() val files = (1 to 3).map(i => new File(tmpDir, i.toString)) - Files.write("0123456789", files(0), Charsets.UTF_8) - Files.write("abcdefghij", files(1), Charsets.UTF_8) - Files.write("ABCDEFGHIJ", files(2), Charsets.UTF_8) + Files.write("0123456789", files(0), UTF_8) + Files.write("abcdefghij", files(1), UTF_8) + Files.write("ABCDEFGHIJ", files(2), UTF_8) // Read first few bytes in the 1st file assert(Utils.offsetBytes(files, 0, 5) === "01234") @@ -339,7 +339,7 @@ class UtilsSuite extends FunSuite { try { System.setProperty("spark.test.fileNameLoadB", "2") Files.write("spark.test.fileNameLoadA true\n" + - "spark.test.fileNameLoadB 1\n", outFile, Charsets.UTF_8) + "spark.test.fileNameLoadB 1\n", outFile, UTF_8) val properties = Utils.getPropertiesFromFile(outFile.getAbsolutePath) properties .filter { case (k, v) => k.startsWith("spark.")} From 4b55482abf899c27da3d55401ad26b4e9247b327 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 28 Oct 2014 14:36:06 -0700 Subject: [PATCH 069/115] [SPARK-3343] [SQL] Add serde support for CTAS Currently, `CTAS` (Create Table As Select) doesn't support specifying the `SerDe` in HQL. This PR will pass down the `ASTNode` into the physical operator `execution.CreateTableAsSelect`, which will extract the `CreateTableDesc` object via Hive `SemanticAnalyzer`. In the meantime, I also update the `HiveMetastoreCatalog.createTable` to optionally support the `CreateTableDesc` for table creation. Author: Cheng Hao Closes #2570 from chenghao-intel/ctas_serde and squashes the following commits: e011ef5 [Cheng Hao] shim for both 0.12 & 0.13.1 cfb3662 [Cheng Hao] revert to hive 0.12 c8a547d [Cheng Hao] Support SerDe properties within CTAS --- .../plans/logical/basicOperators.scala | 8 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 4 +- .../org/apache/spark/sql/QueryTest.scala | 19 ++ .../execution/HiveCompatibilitySuite.scala | 6 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 197 +++++++++++++++--- .../org/apache/spark/sql/hive/HiveQl.scala | 15 +- .../spark/sql/hive/HiveStrategies.scala | 17 +- .../hive/execution/CreateTableAsSelect.scala | 39 ++-- .../org/apache/spark/sql/QueryTest.scala | 19 ++ .../sql/hive/execution/HiveExplainSuite.scala | 37 ++-- .../sql/hive/execution/SQLQuerySuite.scala | 59 ++++++ .../org/apache/spark/sql/hive/Shim12.scala | 5 +- .../org/apache/spark/sql/hive/Shim13.scala | 6 +- 13 files changed, 337 insertions(+), 94 deletions(-) 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 14b03c7445c13..00bdf108a8398 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 @@ -114,11 +114,13 @@ case class InsertIntoTable( } } -case class CreateTableAsSelect( +case class CreateTableAsSelect[T]( databaseName: Option[String], tableName: String, - child: LogicalPlan) extends UnaryNode { - override def output = child.output + child: LogicalPlan, + allowExisting: Boolean, + desc: Option[T] = None) extends UnaryNode { + override def output = Seq.empty[Attribute] override lazy val resolved = (databaseName != None && childrenResolved) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 25ba7d88ba538..15516afb95504 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -54,7 +54,7 @@ private[sql] trait SchemaRDDLike { @transient protected[spark] val logicalPlan: LogicalPlan = baseLogicalPlan match { // For various commands (like DDL) and queries with side effects, we force query optimization to // happen right away to let these side effects take place eagerly. - case _: Command | _: InsertIntoTable | _: CreateTableAsSelect |_: WriteToFile => + case _: Command | _: InsertIntoTable | _: CreateTableAsSelect[_] |_: WriteToFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) case _ => baseLogicalPlan @@ -123,7 +123,7 @@ private[sql] trait SchemaRDDLike { */ @Experimental def saveAsTable(tableName: String): Unit = - sqlContext.executePlan(CreateTableAsSelect(None, tableName, logicalPlan)).toRdd + sqlContext.executePlan(CreateTableAsSelect(None, tableName, logicalPlan, false)).toRdd /** Returns the schema as a string in the tree format. * 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 1fd8d27b34c59..042f61f5a4113 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 @@ -21,6 +21,25 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ 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 rdd the [[SchemaRDD]] 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 + * @param keywords keyword in string array + */ + def checkExistence(rdd: SchemaRDD, exists: Boolean, keywords: String*) { + val outputs = rdd.collect().map(_.mkString).mkString + for (key <- keywords) { + if (exists) { + assert(outputs.contains(key), s"Failed for $rdd ($key doens't exist in result)") + } else { + assert(!outputs.contains(key), s"Failed for $rdd ($key existed in the result)") + } + } + } + /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[SchemaRDD]] to be executed 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 4fc26d6f55511..26d9ca05c851b 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 @@ -229,7 +229,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Needs constant object inspectors "udf_round", - "udf7" + "udf7", + + // Sort with Limit clause causes failure. + "ctas", + "ctas_hadoop20" ) ++ HiveShim.compatibilityBlackList /** 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 39d87a9d14fa7..2dd2c882a8420 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 @@ -17,17 +17,27 @@ package org.apache.spark.sql.hive +import java.io.IOException +import java.util.{List => JList} + import scala.util.parsing.combinator.RegexParsers -import org.apache.hadoop.hive.metastore.api.{FieldSchema, SerDeInfo, StorageDescriptor, Partition => TPartition, Table => TTable} -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} -import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.util.ReflectionUtils +import org.apache.hadoop.fs.Path + +import org.apache.hadoop.hive.metastore.TableType +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table, HiveException} +import org.apache.hadoop.hive.ql.plan.{TableDesc, CreateTableDesc} +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.analysis.Catalog +import org.apache.spark.sql.catalyst.analysis.{Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ @@ -66,37 +76,164 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with table.getTTable, partitions.map(part => part.getTPartition))(hive) } + /** + * Create table with specified database, table name, table description and schema + * @param databaseName Database Name + * @param tableName Table Name + * @param schema Schema of the new table, if not specified, will use the schema + * specified in crtTbl + * @param allowExisting if true, ignore AlreadyExistsException + * @param desc CreateTableDesc object which contains the SerDe info. Currently + * we support most of the features except the bucket. + */ def createTable( databaseName: String, tableName: String, schema: Seq[Attribute], - allowExisting: Boolean = false): Unit = { + allowExisting: Boolean = false, + desc: Option[CreateTableDesc] = None) { + val hconf = hive.hiveconf + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) - val table = new Table(dbName, tblName) - val hiveSchema = + val tbl = new Table(dbName, tblName) + + val crtTbl: CreateTableDesc = desc.getOrElse(null) + + // We should respect the passed in schema, unless it's not set + val hiveSchema: JList[FieldSchema] = if (schema == null || schema.isEmpty) { + crtTbl.getCols + } else { schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) - table.setFields(hiveSchema) - - val sd = new StorageDescriptor() - table.getTTable.setSd(sd) - sd.setCols(hiveSchema) - - // TODO: THESE ARE ALL DEFAULTS, WE NEED TO PARSE / UNDERSTAND the output specs. - sd.setCompressed(false) - sd.setParameters(Map[String, String]()) - sd.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") - sd.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") - val serDeInfo = new SerDeInfo() - serDeInfo.setName(tblName) - serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") - serDeInfo.setParameters(Map[String, String]()) - sd.setSerdeInfo(serDeInfo) + } + tbl.setFields(hiveSchema) + + // Most of code are similar with the DDLTask.createTable() of Hive, + if (crtTbl != null && crtTbl.getTblProps() != null) { + tbl.getTTable().getParameters().putAll(crtTbl.getTblProps()) + } + + if (crtTbl != null && crtTbl.getPartCols() != null) { + tbl.setPartCols(crtTbl.getPartCols()) + } + + if (crtTbl != null && crtTbl.getStorageHandler() != null) { + tbl.setProperty( + org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE, + crtTbl.getStorageHandler()) + } + + /* + * We use LazySimpleSerDe by default. + * + * If the user didn't specify a SerDe, and any of the columns are not simple + * types, we will have to use DynamicSerDe instead. + */ + if (crtTbl == null || crtTbl.getSerName() == null) { + val storageHandler = tbl.getStorageHandler() + if (storageHandler == null) { + logInfo(s"Default to LazySimpleSerDe for table $dbName.$tblName") + tbl.setSerializationLib(classOf[LazySimpleSerDe].getName()) + + import org.apache.hadoop.mapred.TextInputFormat + import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + import org.apache.hadoop.io.Text + + tbl.setInputFormatClass(classOf[TextInputFormat]) + tbl.setOutputFormatClass(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]]) + tbl.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") + } else { + val serDeClassName = storageHandler.getSerDeClass().getName() + logInfo(s"Use StorageHandler-supplied $serDeClassName for table $dbName.$tblName") + tbl.setSerializationLib(serDeClassName) + } + } else { + // let's validate that the serde exists + val serdeName = crtTbl.getSerName() + try { + val d = ReflectionUtils.newInstance(hconf.getClassByName(serdeName), hconf) + if (d != null) { + logDebug("Found class for $serdeName") + } + } catch { + case e: SerDeException => throw new HiveException("Cannot validate serde: " + serdeName, e) + } + tbl.setSerializationLib(serdeName) + } + + if (crtTbl != null && crtTbl.getFieldDelim() != null) { + tbl.setSerdeParam(serdeConstants.FIELD_DELIM, crtTbl.getFieldDelim()) + tbl.setSerdeParam(serdeConstants.SERIALIZATION_FORMAT, crtTbl.getFieldDelim()) + } + if (crtTbl != null && crtTbl.getFieldEscape() != null) { + tbl.setSerdeParam(serdeConstants.ESCAPE_CHAR, crtTbl.getFieldEscape()) + } + + if (crtTbl != null && crtTbl.getCollItemDelim() != null) { + tbl.setSerdeParam(serdeConstants.COLLECTION_DELIM, crtTbl.getCollItemDelim()) + } + if (crtTbl != null && crtTbl.getMapKeyDelim() != null) { + tbl.setSerdeParam(serdeConstants.MAPKEY_DELIM, crtTbl.getMapKeyDelim()) + } + if (crtTbl != null && crtTbl.getLineDelim() != null) { + tbl.setSerdeParam(serdeConstants.LINE_DELIM, crtTbl.getLineDelim()) + } + + if (crtTbl != null && crtTbl.getSerdeProps() != null) { + val iter = crtTbl.getSerdeProps().entrySet().iterator() + while (iter.hasNext()) { + val m = iter.next() + tbl.setSerdeParam(m.getKey(), m.getValue()) + } + } + + if (crtTbl != null && crtTbl.getComment() != null) { + tbl.setProperty("comment", crtTbl.getComment()) + } + + if (crtTbl != null && crtTbl.getLocation() != null) { + HiveShim.setLocation(tbl, crtTbl) + } + + if (crtTbl != null && crtTbl.getSkewedColNames() != null) { + tbl.setSkewedColNames(crtTbl.getSkewedColNames()) + } + if (crtTbl != null && crtTbl.getSkewedColValues() != null) { + tbl.setSkewedColValues(crtTbl.getSkewedColValues()) + } + + if (crtTbl != null) { + tbl.setStoredAsSubDirectories(crtTbl.isStoredAsSubDirectories()) + tbl.setInputFormatClass(crtTbl.getInputFormat()) + tbl.setOutputFormatClass(crtTbl.getOutputFormat()) + } + + tbl.getTTable().getSd().setInputFormat(tbl.getInputFormatClass().getName()) + tbl.getTTable().getSd().setOutputFormat(tbl.getOutputFormatClass().getName()) + + if (crtTbl != null && crtTbl.isExternal()) { + tbl.setProperty("EXTERNAL", "TRUE") + tbl.setTableType(TableType.EXTERNAL_TABLE) + } + + // set owner + try { + tbl.setOwner(hive.hiveconf.getUser) + } catch { + case e: IOException => throw new HiveException("Unable to get current user", e) + } + + // set create time + tbl.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) + + // TODO add bucket support + // TODO set more info if Hive upgrade + // create the table synchronized { - try client.createTable(table) catch { - case e: org.apache.hadoop.hive.ql.metadata.HiveException - if e.getCause.isInstanceOf[org.apache.hadoop.hive.metastore.api.AlreadyExistsException] && - allowExisting => // Do nothing. + try client.createTable(tbl, allowExisting) catch { + case e: org.apache.hadoop.hive.metastore.api.AlreadyExistsException + if allowExisting => // Do nothing + case e: Throwable => throw e } } } @@ -110,11 +247,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case CreateTableAsSelect(db, tableName, child) => + case CreateTableAsSelect(db, tableName, child, allowExisting, extra) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - CreateTableAsSelect(Some(databaseName), tableName, child) + CreateTableAsSelect(Some(databaseName), tableName, child, allowExisting, extra) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index ed07a280398bd..9d9d68affa54b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -448,14 +448,14 @@ private[hive] object HiveQl { } case Token("TOK_CREATETABLE", children) - if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => - // TODO: Parse other clauses. + if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty => // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL val ( Some(tableNameParts) :: _ /* likeTable */ :: - Some(query) +: - notImplemented) = + Some(query) :: + allowExisting +: + ignores) = getClauses( Seq( "TOK_TABNAME", @@ -479,14 +479,9 @@ private[hive] object HiveQl { "TOK_TABLELOCATION", "TOK_TABLEPROPERTIES"), children) - if (notImplemented.exists(token => !token.isEmpty)) { - throw new NotImplementedError( - s"Unhandled clauses: ${notImplemented.flatten.map(dumpTree(_)).mkString("\n")}") - } - val (db, tableName) = extractDbNameTableName(tableNameParts) - CreateTableAsSelect(db, tableName, nodeToPlan(query)) + CreateTableAsSelect(db, tableName, nodeToPlan(query), allowExisting != None, Some(node)) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder 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 5c66322f1ed99..e59d4d536a0af 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.ql.parse.ASTNode + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ @@ -160,17 +162,14 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil - - case logical.CreateTableAsSelect(database, tableName, child) => - val query = planLater(child) + case logical.CreateTableAsSelect( + Some(database), tableName, child, allowExisting, Some(extra: ASTNode)) => CreateTableAsSelect( - database.get, + database, tableName, - query, - InsertIntoHiveTable(_: MetastoreRelation, - Map(), - query, - overwrite = true)(hiveContext)) :: Nil + child, + allowExisting, + extra) :: Nil case _ => Nil } } 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 3625708d03175..2fce414734579 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 @@ -17,10 +17,12 @@ package org.apache.spark.sql.hive.execution +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.parse.{SemanticAnalyzer, ASTNode} import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.MetastoreRelation @@ -30,33 +32,46 @@ import org.apache.spark.sql.hive.MetastoreRelation * Create table and insert the query result into it. * @param database the database name of the new relation * @param tableName the table name of the new relation - * @param insertIntoRelation function of creating the `InsertIntoHiveTable` - * by specifying the `MetaStoreRelation`, the data will be inserted into that table. - * TODO Add more table creating properties, e.g. SerDe, StorageHandler, in-memory cache etc. + * @param query the query whose result will be insert into the new relation + * @param allowExisting allow continue working if it's already exists, otherwise + * raise exception + * @param extra the extra information for this Operator, it should be the + * ASTNode object for extracting the CreateTableDesc. + */ @Experimental case class CreateTableAsSelect( database: String, tableName: String, - query: SparkPlan, - insertIntoRelation: MetastoreRelation => InsertIntoHiveTable) - extends LeafNode with Command { + query: LogicalPlan, + allowExisting: Boolean, + extra: ASTNode) extends LeafNode with Command { def output = Seq.empty + private[this] def sc = sqlContext.asInstanceOf[HiveContext] + // A lazy computing of the metastoreRelation private[this] lazy val metastoreRelation: MetastoreRelation = { - // Create the table - val sc = sqlContext.asInstanceOf[HiveContext] - sc.catalog.createTable(database, tableName, query.output, false) + // Get the CreateTableDesc from Hive SemanticAnalyzer + val sa = new SemanticAnalyzer(sc.hiveconf) + + sa.analyze(extra, new Context(sc.hiveconf)) + val desc = sa.getQB().getTableDesc + // Create Hive Table + sc.catalog.createTable(database, tableName, query.output, allowExisting, Some(desc)) + // Get the Metastore Relation sc.catalog.lookupRelation(Some(database), tableName, None) match { case r: MetastoreRelation => r } } - override protected lazy val sideEffectResult: Seq[Row] = { - insertIntoRelation(metastoreRelation).execute + override protected[sql] lazy val sideEffectResult: Seq[Row] = { + // 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. + sc.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd Seq.empty[Row] } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index 6b0641052052d..f89c49d292c6c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -32,6 +32,25 @@ import org.apache.spark.sql.catalyst.util._ * So, we duplicate this code here. */ 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 rdd the [[SchemaRDD]] 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 + * @param keywords keyword in string array + */ + def checkExistence(rdd: SchemaRDD, exists: Boolean, keywords: String*) { + val outputs = rdd.collect().map(_.mkString).mkString + for (key <- keywords) { + if (exists) { + assert(outputs.contains(key), s"Failed for $rdd ($key doens't exist in result)") + } else { + assert(!outputs.contains(key), s"Failed for $rdd ($key existed in the result)") + } + } + } + /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[SchemaRDD]] to be executed diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 4ed58f4be1167..a68fc2a803bb4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -18,37 +18,24 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.Row /** * A set of tests that validates support for Hive Explain command. */ class HiveExplainSuite extends QueryTest { - private def check(sqlCmd: String, exists: Boolean, keywords: String*) { - val outputs = sql(sqlCmd).collect().map(_.getString(0)).mkString - for (key <- keywords) { - if (exists) { - assert(outputs.contains(key), s"Failed for $sqlCmd ($key doens't exist in result)") - } else { - assert(!outputs.contains(key), s"Failed for $sqlCmd ($key existed in the result)") - } - } - } - test("explain extended command") { - check(" explain select * from src where key=123 ", true, - "== Physical Plan ==") - check(" explain select * from src where key=123 ", false, - "== Parsed Logical Plan ==", - "== Analyzed Logical Plan ==", - "== Optimized Logical Plan ==") - check(" explain extended select * from src where key=123 ", true, - "== Parsed Logical Plan ==", - "== Analyzed Logical Plan ==", - "== Optimized Logical Plan ==", - "== Physical Plan ==", - "Code Generation", "== RDD ==") + checkExistence(sql(" explain select * from src where key=123 "), true, + "== Physical Plan ==") + checkExistence(sql(" explain select * from src where key=123 "), false, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==") + checkExistence(sql(" explain extended select * from src where key=123 "), true, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==", + "== Physical Plan ==", + "Code Generation", "== RDD ==") } } 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 a4aea31d3ff9e..4f96a327ee2c7 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 @@ -32,6 +32,65 @@ case class Nested3(f3: Int) * valid, but Hive currently cannot execute it. */ class SQLQuerySuite extends QueryTest { + test("CTAS with serde") { + sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect + sql( + """CREATE TABLE ctas2 + | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" + | WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") + | STORED AS RCFile + | TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") + | AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin).collect + sql( + """CREATE TABLE ctas3 + | ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\012' + | STORED AS textfile AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin).collect + + // the table schema may like (key: integer, value: string) + sql( + """CREATE TABLE IF NOT EXISTS ctas4 AS + | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin).collect + // expect the string => integer for field key cause the table ctas4 already existed. + sql( + """CREATE TABLE IF NOT EXISTS ctas4 AS + | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect + + checkAnswer( + sql("SELECT k, value FROM ctas1 ORDER BY k, value"), + sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq) + checkAnswer( + sql("SELECT key, value FROM ctas2 ORDER BY key, value"), + sql( + """ + SELECT key, value + FROM src + ORDER BY key, value""").collect().toSeq) + checkAnswer( + sql("SELECT key, value FROM ctas3 ORDER BY key, value"), + sql( + """ + SELECT key, value + FROM src + ORDER BY key, value""").collect().toSeq) + checkAnswer( + sql("SELECT key, value FROM ctas4 ORDER BY key, value"), + sql("SELECT CAST(key AS int) k, value FROM src ORDER BY k, value").collect().toSeq) + + checkExistence(sql("DESC EXTENDED ctas2"), true, + "name:key", "type:string", "name:value", "ctas2", + "org.apache.hadoop.hive.ql.io.RCFileInputFormat", + "org.apache.hadoop.hive.ql.io.RCFileOutputFormat", + "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe", + "serde_p1=p1", "serde_p2=p2", "tbl_p1=p11", "tbl_p2=p22","MANAGED_TABLE" + ) + } + test("ordering not in select") { checkAnswer( sql("SELECT key FROM src ORDER BY value"), diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 2317d2e76341f..8cb81db8a9724 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} @@ -89,6 +89,9 @@ private[hive] object HiveShim { "udf_concat" ) + def setLocation(tbl: Table, crtTbl: CreateTableDesc): Unit = { + tbl.setDataLocation(new Path(crtTbl.getLocation()).toUri()) + } } class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index b8d893d8c1319..b9a742cc6ef67 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.common.`type`.{HiveDecimal} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} import org.apache.hadoop.mapred.InputFormat @@ -121,6 +121,10 @@ private[hive] object HiveShim { def compatibilityBlackList = Seq() + def setLocation(tbl: Table, crtTbl: CreateTableDesc): Unit = { + tbl.setDataLocation(new Path(crtTbl.getLocation())) + } + /* * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not. * Fix it through wrapper. From 84e5da87e32256ba4f3dee6f8bf532ce88322028 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 28 Oct 2014 15:14:41 -0700 Subject: [PATCH 070/115] [SPARK-4084] Reuse sort key in Sorter Sorter uses generic-typed key for sorting. When data is large, it creates lots of key objects, which is not efficient. We should reuse the key in Sorter for memory efficiency. This change is part of the petabyte sort implementation from rxin . The `Sorter` class was written in Java and marked package private. So it is only available to `org.apache.spark.util.collection`. I renamed it to `TimSort` and add a simple wrapper of it, still called `Sorter`, in Scala, which is `private[spark]`. The benchmark code is updated, which now resets the array before each run. Here is the result on sorting primitive Int arrays of size 25 million using Sorter: ~~~ [info] - Sorter benchmark for key-value pairs !!! IGNORED !!! Java Arrays.sort() on non-primitive int array: Took 13237 ms Java Arrays.sort() on non-primitive int array: Took 13320 ms Java Arrays.sort() on non-primitive int array: Took 15718 ms Java Arrays.sort() on non-primitive int array: Took 13283 ms Java Arrays.sort() on non-primitive int array: Took 13267 ms Java Arrays.sort() on non-primitive int array: Took 15122 ms Java Arrays.sort() on non-primitive int array: Took 15495 ms Java Arrays.sort() on non-primitive int array: Took 14877 ms Java Arrays.sort() on non-primitive int array: Took 16429 ms Java Arrays.sort() on non-primitive int array: Took 14250 ms Java Arrays.sort() on non-primitive int array: (13878 ms first try, 14499 ms average) Java Arrays.sort() on primitive int array: Took 2683 ms Java Arrays.sort() on primitive int array: Took 2683 ms Java Arrays.sort() on primitive int array: Took 2701 ms Java Arrays.sort() on primitive int array: Took 2746 ms Java Arrays.sort() on primitive int array: Took 2685 ms Java Arrays.sort() on primitive int array: Took 2735 ms Java Arrays.sort() on primitive int array: Took 2669 ms Java Arrays.sort() on primitive int array: Took 2693 ms Java Arrays.sort() on primitive int array: Took 2680 ms Java Arrays.sort() on primitive int array: Took 2642 ms Java Arrays.sort() on primitive int array: (2948 ms first try, 2691 ms average) Sorter without key reuse on primitive int array: Took 10732 ms Sorter without key reuse on primitive int array: Took 12482 ms Sorter without key reuse on primitive int array: Took 10718 ms Sorter without key reuse on primitive int array: Took 12650 ms Sorter without key reuse on primitive int array: Took 10747 ms Sorter without key reuse on primitive int array: Took 10783 ms Sorter without key reuse on primitive int array: Took 12721 ms Sorter without key reuse on primitive int array: Took 10604 ms Sorter without key reuse on primitive int array: Took 10622 ms Sorter without key reuse on primitive int array: Took 11843 ms Sorter without key reuse on primitive int array: (11089 ms first try, 11390 ms average) Sorter with key reuse on primitive int array: Took 5141 ms Sorter with key reuse on primitive int array: Took 5298 ms Sorter with key reuse on primitive int array: Took 5066 ms Sorter with key reuse on primitive int array: Took 5164 ms Sorter with key reuse on primitive int array: Took 5203 ms Sorter with key reuse on primitive int array: Took 5274 ms Sorter with key reuse on primitive int array: Took 5186 ms Sorter with key reuse on primitive int array: Took 5159 ms Sorter with key reuse on primitive int array: Took 5164 ms Sorter with key reuse on primitive int array: Took 5078 ms Sorter with key reuse on primitive int array: (5311 ms first try, 5173 ms average) ~~~ So with key reuse, it is faster and less likely to trigger GC. Author: Xiangrui Meng Author: Reynold Xin Closes #2937 from mengxr/SPARK-4084 and squashes the following commits: d73c3d0 [Xiangrui Meng] address comments 0b7b682 [Xiangrui Meng] fix mima a72f53c [Xiangrui Meng] update timeIt 38ba50c [Xiangrui Meng] update timeIt 720f731 [Xiangrui Meng] add doc about JIT specialization 78f2879 [Xiangrui Meng] update tests 7de2efd [Xiangrui Meng] update the Sorter benchmark code to be correct 8626356 [Xiangrui Meng] add prepare to timeIt and update testsin SorterSuite 5f0d530 [Xiangrui Meng] update method modifiers of SortDataFormat 6ffbe66 [Xiangrui Meng] rename Sorter to TimSort and add a Scala wrapper that is private[spark] b00db4d [Xiangrui Meng] doc and tests cf94e8a [Xiangrui Meng] renaming 464ddce [Reynold Xin] cherry-pick rxin's commit --- .../collection/{Sorter.java => TimSort.java} | 77 ++++--- .../scala/org/apache/spark/util/Utils.scala | 26 ++- .../util/collection/SortDataFormat.scala | 41 +++- .../apache/spark/util/collection/Sorter.scala | 39 ++++ .../spark/util/random/XORShiftRandom.scala | 8 +- .../org/apache/spark/util/UtilsSuite.scala | 11 + .../spark/util/collection/SorterSuite.scala | 210 +++++++++++++----- project/MimaExcludes.scala | 4 +- 8 files changed, 310 insertions(+), 106 deletions(-) rename core/src/main/java/org/apache/spark/util/collection/{Sorter.java => TimSort.java} (92%) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/Sorter.scala diff --git a/core/src/main/java/org/apache/spark/util/collection/Sorter.java b/core/src/main/java/org/apache/spark/util/collection/TimSort.java similarity index 92% rename from core/src/main/java/org/apache/spark/util/collection/Sorter.java rename to core/src/main/java/org/apache/spark/util/collection/TimSort.java index 64ad18c0e463a..409e1a41c5d49 100644 --- a/core/src/main/java/org/apache/spark/util/collection/Sorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/TimSort.java @@ -20,18 +20,25 @@ import java.util.Comparator; /** - * A port of the Android Timsort class, which utilizes a "stable, adaptive, iterative mergesort." + * A port of the Android TimSort class, which utilizes a "stable, adaptive, iterative mergesort." * See the method comment on sort() for more details. * * This has been kept in Java with the original style in order to match very closely with the - * Anroid source code, and thus be easy to verify correctness. + * Android source code, and thus be easy to verify correctness. The class is package private. We put + * a simple Scala wrapper {@link org.apache.spark.util.collection.Sorter}, which is available to + * package org.apache.spark. * * The purpose of the port is to generalize the interface to the sort to accept input data formats * besides simple arrays where every element is sorted individually. For instance, the AppendOnlyMap * uses this to sort an Array with alternating elements of the form [key, value, key, value]. * This generalization comes with minimal overhead -- see SortDataFormat for more information. + * + * We allow key reuse to prevent creating many key objects -- see SortDataFormat. + * + * @see org.apache.spark.util.collection.SortDataFormat + * @see org.apache.spark.util.collection.Sorter */ -class Sorter { +class TimSort { /** * This is the minimum sized sequence that will be merged. Shorter @@ -54,7 +61,7 @@ class Sorter { private final SortDataFormat s; - public Sorter(SortDataFormat sortDataFormat) { + public TimSort(SortDataFormat sortDataFormat) { this.s = sortDataFormat; } @@ -91,7 +98,7 @@ public Sorter(SortDataFormat sortDataFormat) { * * @author Josh Bloch */ - void sort(Buffer a, int lo, int hi, Comparator c) { + public void sort(Buffer a, int lo, int hi, Comparator c) { assert c != null; int nRemaining = hi - lo; @@ -162,10 +169,13 @@ private void binarySort(Buffer a, int lo, int hi, int start, Comparator>> 1; - if (c.compare(pivot, s.getKey(a, mid)) < 0) + if (c.compare(pivot, s.getKey(a, mid, key1)) < 0) right = mid; else left = mid + 1; @@ -235,13 +245,16 @@ private int countRunAndMakeAscending(Buffer a, int lo, int hi, Comparator= 0) + while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) >= 0) runHi++; } @@ -468,11 +481,13 @@ private void mergeAt(int i) { } stackSize--; + K key0 = s.newKey(); + /* * Find where the first element of run2 goes in run1. Prior elements * in run1 can be ignored (because they're already in place). */ - int k = gallopRight(s.getKey(a, base2), a, base1, len1, 0, c); + int k = gallopRight(s.getKey(a, base2, key0), a, base1, len1, 0, c); assert k >= 0; base1 += k; len1 -= k; @@ -483,7 +498,7 @@ private void mergeAt(int i) { * Find where the last element of run1 goes in run2. Subsequent elements * in run2 can be ignored (because they're already in place). */ - len2 = gallopLeft(s.getKey(a, base1 + len1 - 1), a, base2, len2, len2 - 1, c); + len2 = gallopLeft(s.getKey(a, base1 + len1 - 1, key0), a, base2, len2, len2 - 1, c); assert len2 >= 0; if (len2 == 0) return; @@ -517,10 +532,12 @@ private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator< assert len > 0 && hint >= 0 && hint < len; int lastOfs = 0; int ofs = 1; - if (c.compare(key, s.getKey(a, base + hint)) > 0) { + K key0 = s.newKey(); + + if (c.compare(key, s.getKey(a, base + hint, key0)) > 0) { // Gallop right until a[base+hint+lastOfs] < key <= a[base+hint+ofs] int maxOfs = len - hint; - while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs)) > 0) { + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key0)) > 0) { lastOfs = ofs; ofs = (ofs << 1) + 1; if (ofs <= 0) // int overflow @@ -535,7 +552,7 @@ private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator< } else { // key <= a[base + hint] // Gallop left until a[base+hint-ofs] < key <= a[base+hint-lastOfs] final int maxOfs = hint + 1; - while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs)) <= 0) { + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key0)) <= 0) { lastOfs = ofs; ofs = (ofs << 1) + 1; if (ofs <= 0) // int overflow @@ -560,7 +577,7 @@ private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator< while (lastOfs < ofs) { int m = lastOfs + ((ofs - lastOfs) >>> 1); - if (c.compare(key, s.getKey(a, base + m)) > 0) + if (c.compare(key, s.getKey(a, base + m, key0)) > 0) lastOfs = m + 1; // a[base + m] < key else ofs = m; // key <= a[base + m] @@ -587,10 +604,12 @@ private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator int ofs = 1; int lastOfs = 0; - if (c.compare(key, s.getKey(a, base + hint)) < 0) { + K key1 = s.newKey(); + + if (c.compare(key, s.getKey(a, base + hint, key1)) < 0) { // Gallop left until a[b+hint - ofs] <= key < a[b+hint - lastOfs] int maxOfs = hint + 1; - while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs)) < 0) { + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key1)) < 0) { lastOfs = ofs; ofs = (ofs << 1) + 1; if (ofs <= 0) // int overflow @@ -606,7 +625,7 @@ private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator } else { // a[b + hint] <= key // Gallop right until a[b+hint + lastOfs] <= key < a[b+hint + ofs] int maxOfs = len - hint; - while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs)) >= 0) { + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key1)) >= 0) { lastOfs = ofs; ofs = (ofs << 1) + 1; if (ofs <= 0) // int overflow @@ -630,7 +649,7 @@ private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator while (lastOfs < ofs) { int m = lastOfs + ((ofs - lastOfs) >>> 1); - if (c.compare(key, s.getKey(a, base + m)) < 0) + if (c.compare(key, s.getKey(a, base + m, key1)) < 0) ofs = m; // key < a[b + m] else lastOfs = m + 1; // a[b + m] <= key @@ -679,6 +698,9 @@ private void mergeLo(int base1, int len1, int base2, int len2) { return; } + K key0 = s.newKey(); + K key1 = s.newKey(); + Comparator c = this.c; // Use local variable for performance int minGallop = this.minGallop; // " " " " " outer: @@ -692,7 +714,7 @@ private void mergeLo(int base1, int len1, int base2, int len2) { */ do { assert len1 > 1 && len2 > 0; - if (c.compare(s.getKey(a, cursor2), s.getKey(tmp, cursor1)) < 0) { + if (c.compare(s.getKey(a, cursor2, key0), s.getKey(tmp, cursor1, key1)) < 0) { s.copyElement(a, cursor2++, a, dest++); count2++; count1 = 0; @@ -714,7 +736,7 @@ private void mergeLo(int base1, int len1, int base2, int len2) { */ do { assert len1 > 1 && len2 > 0; - count1 = gallopRight(s.getKey(a, cursor2), tmp, cursor1, len1, 0, c); + count1 = gallopRight(s.getKey(a, cursor2, key0), tmp, cursor1, len1, 0, c); if (count1 != 0) { s.copyRange(tmp, cursor1, a, dest, count1); dest += count1; @@ -727,7 +749,7 @@ private void mergeLo(int base1, int len1, int base2, int len2) { if (--len2 == 0) break outer; - count2 = gallopLeft(s.getKey(tmp, cursor1), a, cursor2, len2, 0, c); + count2 = gallopLeft(s.getKey(tmp, cursor1, key0), a, cursor2, len2, 0, c); if (count2 != 0) { s.copyRange(a, cursor2, a, dest, count2); dest += count2; @@ -784,6 +806,9 @@ private void mergeHi(int base1, int len1, int base2, int len2) { int cursor2 = len2 - 1; // Indexes into tmp array int dest = base2 + len2 - 1; // Indexes into a + K key0 = s.newKey(); + K key1 = s.newKey(); + // Move last element of first run and deal with degenerate cases s.copyElement(a, cursor1--, a, dest--); if (--len1 == 0) { @@ -811,7 +836,7 @@ private void mergeHi(int base1, int len1, int base2, int len2) { */ do { assert len1 > 0 && len2 > 1; - if (c.compare(s.getKey(tmp, cursor2), s.getKey(a, cursor1)) < 0) { + if (c.compare(s.getKey(tmp, cursor2, key0), s.getKey(a, cursor1, key1)) < 0) { s.copyElement(a, cursor1--, a, dest--); count1++; count2 = 0; @@ -833,7 +858,7 @@ private void mergeHi(int base1, int len1, int base2, int len2) { */ do { assert len1 > 0 && len2 > 1; - count1 = len1 - gallopRight(s.getKey(tmp, cursor2), a, base1, len1, len1 - 1, c); + count1 = len1 - gallopRight(s.getKey(tmp, cursor2, key0), a, base1, len1, len1 - 1, c); if (count1 != 0) { dest -= count1; cursor1 -= count1; @@ -846,7 +871,7 @@ private void mergeHi(int base1, int len1, int base2, int len2) { if (--len2 == 1) break outer; - count2 = len2 - gallopLeft(s.getKey(a, cursor1), tmp, 0, len2, len2 - 1, c); + count2 = len2 - gallopLeft(s.getKey(a, cursor1, key0), tmp, 0, len2, len2 - 1, c); if (count2 != 0) { dest -= count2; cursor2 -= count2; 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 612eca308bf0b..1e881da5114d3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1272,12 +1272,28 @@ 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 + * @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. + * @param prepare function to be executed before each call to f. Its running time doesn't count. + * @return the total time across all iterations (not couting preparation time) */ - def timeIt(numIters: Int)(f: => Unit): Long = { - val start = System.currentTimeMillis - times(numIters)(f) - System.currentTimeMillis - start + def timeIt(numIters: Int)(f: => Unit, prepare: Option[() => Unit] = None): Long = { + if (prepare.isEmpty) { + val start = System.currentTimeMillis + times(numIters)(f) + System.currentTimeMillis - start + } else { + var i = 0 + var sum = 0L + while (i < numIters) { + prepare.get.apply() + val start = System.currentTimeMillis + f + sum += System.currentTimeMillis - start + i += 1 + } + sum + } } /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala b/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala index ac1528969f0be..4f0bf8384afc9 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala @@ -27,33 +27,51 @@ import scala.reflect.ClassTag * Example format: an array of numbers, where each element is also the key. * See [[KVArraySortDataFormat]] for a more exciting format. * - * This trait extends Any to ensure it is universal (and thus compiled to a Java interface). + * Note: Declaring and instantiating multiple subclasses of this class would prevent JIT inlining + * overridden methods and hence decrease the shuffle performance. * * @tparam K Type of the sort key of each element * @tparam Buffer Internal data structure used by a particular format (e.g., Array[Int]). */ // TODO: Making Buffer a real trait would be a better abstraction, but adds some complexity. -private[spark] trait SortDataFormat[K, Buffer] extends Any { +private[spark] +abstract class SortDataFormat[K, Buffer] { + + /** + * Creates a new mutable key for reuse. This should be implemented if you want to override + * [[getKey(Buffer, Int, K)]]. + */ + def newKey(): K = null.asInstanceOf[K] + /** Return the sort key for the element at the given index. */ protected def getKey(data: Buffer, pos: Int): K + /** + * Returns the sort key for the element at the given index and reuse the input key if possible. + * The default implementation ignores the reuse parameter and invokes [[getKey(Buffer, Int]]. + * If you want to override this method, you must implement [[newKey()]]. + */ + def getKey(data: Buffer, pos: Int, reuse: K): K = { + getKey(data, pos) + } + /** Swap two elements. */ - protected def swap(data: Buffer, pos0: Int, pos1: Int): Unit + def swap(data: Buffer, pos0: Int, pos1: Int): Unit /** Copy a single element from src(srcPos) to dst(dstPos). */ - protected def copyElement(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int): Unit + def copyElement(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int): Unit /** * Copy a range of elements starting at src(srcPos) to dst, starting at dstPos. * Overlapping ranges are allowed. */ - protected def copyRange(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int, length: Int): Unit + def copyRange(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int, length: Int): Unit /** * Allocates a Buffer that can hold up to 'length' elements. * All elements of the buffer should be considered invalid until data is explicitly copied in. */ - protected def allocate(length: Int): Buffer + def allocate(length: Int): Buffer } /** @@ -67,9 +85,9 @@ private[spark] trait SortDataFormat[K, Buffer] extends Any { private[spark] class KVArraySortDataFormat[K, T <: AnyRef : ClassTag] extends SortDataFormat[K, Array[T]] { - override protected def getKey(data: Array[T], pos: Int): K = data(2 * pos).asInstanceOf[K] + override def getKey(data: Array[T], pos: Int): K = data(2 * pos).asInstanceOf[K] - override protected def swap(data: Array[T], pos0: Int, pos1: Int) { + override def swap(data: Array[T], pos0: Int, pos1: Int) { val tmpKey = data(2 * pos0) val tmpVal = data(2 * pos0 + 1) data(2 * pos0) = data(2 * pos1) @@ -78,17 +96,16 @@ class KVArraySortDataFormat[K, T <: AnyRef : ClassTag] extends SortDataFormat[K, data(2 * pos1 + 1) = tmpVal } - override protected def copyElement(src: Array[T], srcPos: Int, dst: Array[T], dstPos: Int) { + override def copyElement(src: Array[T], srcPos: Int, dst: Array[T], dstPos: Int) { dst(2 * dstPos) = src(2 * srcPos) dst(2 * dstPos + 1) = src(2 * srcPos + 1) } - override protected def copyRange(src: Array[T], srcPos: Int, - dst: Array[T], dstPos: Int, length: Int) { + override def copyRange(src: Array[T], srcPos: Int, dst: Array[T], dstPos: Int, length: Int) { System.arraycopy(src, 2 * srcPos, dst, 2 * dstPos, 2 * length) } - override protected def allocate(length: Int): Array[T] = { + override def allocate(length: Int): Array[T] = { new Array[T](2 * length) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/Sorter.scala b/core/src/main/scala/org/apache/spark/util/collection/Sorter.scala new file mode 100644 index 0000000000000..39f66b8c428c6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/Sorter.scala @@ -0,0 +1,39 @@ +/* + * 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.util.collection + +import java.util.Comparator + +/** + * A simple wrapper over the Java implementation [[TimSort]]. + * + * The Java implementation is package private, and hence it cannot be called outside package + * org.apache.spark.util.collection. This is a simple wrapper of it that is available to spark. + */ +private[spark] +class Sorter[K, Buffer](private val s: SortDataFormat[K, Buffer]) { + + private val timSort = new TimSort(s) + + /** + * Sorts the input buffer within range [lo, hi). + */ + def sort(a: Buffer, lo: Int, hi: Int, c: Comparator[_ >: K]): Unit = { + timSort.sort(a, lo, hi, c) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 55b5713706178..467b890fb4bb9 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -96,13 +96,9 @@ private[spark] object XORShiftRandom { xorRand.nextInt() } - val iters = timeIt(numIters)(_) - /* Return results as a map instead of just printing to screen in case the user wants to do something with them */ - Map("javaTime" -> iters {javaRand.nextInt()}, - "xorTime" -> iters {xorRand.nextInt()}) - + Map("javaTime" -> timeIt(numIters) { javaRand.nextInt() }, + "xorTime" -> timeIt(numIters) { xorRand.nextInt() }) } - } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 65579bb9affc7..1c112334cc2f5 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -351,4 +351,15 @@ class UtilsSuite extends FunSuite { outFile.delete() } } + + test("timeIt with prepare") { + var cnt = 0 + val prepare = () => { + cnt += 1 + Thread.sleep(1000) + } + val time = Utils.timeIt(2)({}, Some(prepare)) + require(cnt === 2, "prepare should be called twice") + require(time < 500, "preparation time should not count") + } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index 6fe1079c2719a..066d47c46a0d2 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.collection -import java.lang.{Float => JFloat} +import java.lang.{Float => JFloat, Integer => JInteger} import java.util.{Arrays, Comparator} import org.scalatest.FunSuite @@ -30,11 +30,15 @@ class SorterSuite extends FunSuite { val rand = new XORShiftRandom(123) val data0 = Array.tabulate[Int](10000) { i => rand.nextInt() } val data1 = data0.clone() + val data2 = data0.clone() Arrays.sort(data0) new Sorter(new IntArraySortDataFormat).sort(data1, 0, data1.length, Ordering.Int) + new Sorter(new KeyReuseIntArraySortDataFormat) + .sort(data2, 0, data2.length, Ordering[IntWrapper]) - data0.zip(data1).foreach { case (x, y) => assert(x === y) } + assert(data0.view === data1.view) + assert(data0.view === data2.view) } test("KVArraySorter") { @@ -61,10 +65,33 @@ class SorterSuite extends FunSuite { } } + /** Runs an experiment several times. */ + def runExperiment(name: String, skip: Boolean = false)(f: => Unit, prepare: () => Unit): Unit = { + if (skip) { + println(s"Skipped experiment $name.") + return + } + + val firstTry = org.apache.spark.util.Utils.timeIt(1)(f, Some(prepare)) + System.gc() + + var i = 0 + var next10: Long = 0 + while (i < 10) { + val time = org.apache.spark.util.Utils.timeIt(1)(f, Some(prepare)) + next10 += time + println(s"$name: Took $time ms") + i += 1 + } + + println(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)") + } + /** * This provides a simple benchmark for comparing the Sorter with Java internal sorting. * Ideally these would be executed one at a time, each in their own JVM, so their listing - * here is mainly to have the code. + * here is mainly to have the code. Running multiple tests within the same JVM session would + * prevent JIT inlining overridden methods and hence hurt the performance. * * The goal of this code is to sort an array of key-value pairs, where the array physically * has the keys and values alternating. The basic Java sorts work only on the keys, so the @@ -72,96 +99,167 @@ class SorterSuite extends FunSuite { * those, while the Sorter approach can work directly on the input data format. * * Note that the Java implementation varies tremendously between Java 6 and Java 7, when - * the Java sort changed from merge sort to Timsort. + * the Java sort changed from merge sort to TimSort. */ - ignore("Sorter benchmark") { - - /** Runs an experiment several times. */ - def runExperiment(name: String)(f: => Unit): Unit = { - val firstTry = org.apache.spark.util.Utils.timeIt(1)(f) - System.gc() - - var i = 0 - var next10: Long = 0 - while (i < 10) { - val time = org.apache.spark.util.Utils.timeIt(1)(f) - next10 += time - println(s"$name: Took $time ms") - i += 1 - } - - println(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)") - } - + ignore("Sorter benchmark for key-value pairs") { val numElements = 25000000 // 25 mil val rand = new XORShiftRandom(123) - val keys = Array.tabulate[JFloat](numElements) { i => - new JFloat(rand.nextFloat()) + // Test our key-value pairs where each element is a Tuple2[Float, Integer]. + + val kvTuples = Array.tabulate(numElements) { i => + (new JFloat(rand.nextFloat()), new JInteger(i)) } - // Test our key-value pairs where each element is a Tuple2[Float, Integer) - val kvTupleArray = Array.tabulate[AnyRef](numElements) { i => - (keys(i / 2): Float, i / 2: Int) + val kvTupleArray = new Array[AnyRef](numElements) + val prepareKvTupleArray = () => { + System.arraycopy(kvTuples, 0, kvTupleArray, 0, numElements) } - runExperiment("Tuple-sort using Arrays.sort()") { + runExperiment("Tuple-sort using Arrays.sort()")({ Arrays.sort(kvTupleArray, new Comparator[AnyRef] { override def compare(x: AnyRef, y: AnyRef): Int = - Ordering.Float.compare(x.asInstanceOf[(Float, _)]._1, y.asInstanceOf[(Float, _)]._1) + x.asInstanceOf[(JFloat, _)]._1.compareTo(y.asInstanceOf[(JFloat, _)]._1) }) - } + }, prepareKvTupleArray) // Test our Sorter where each element alternates between Float and Integer, non-primitive - val keyValueArray = Array.tabulate[AnyRef](numElements * 2) { i => - if (i % 2 == 0) keys(i / 2) else new Integer(i / 2) + + val keyValues = { + val data = new Array[AnyRef](numElements * 2) + var i = 0 + while (i < numElements) { + data(2 * i) = kvTuples(i)._1 + data(2 * i + 1) = kvTuples(i)._2 + i += 1 + } + data } + + val keyValueArray = new Array[AnyRef](numElements * 2) + val prepareKeyValueArray = () => { + System.arraycopy(keyValues, 0, keyValueArray, 0, numElements * 2) + } + val sorter = new Sorter(new KVArraySortDataFormat[JFloat, AnyRef]) - runExperiment("KV-sort using Sorter") { - sorter.sort(keyValueArray, 0, keys.length, new Comparator[JFloat] { - override def compare(x: JFloat, y: JFloat): Int = Ordering.Float.compare(x, y) + runExperiment("KV-sort using Sorter")({ + sorter.sort(keyValueArray, 0, numElements, new Comparator[JFloat] { + override def compare(x: JFloat, y: JFloat): Int = x.compareTo(y) }) + }, prepareKeyValueArray) + } + + /** + * Tests for sorting with primitive keys with/without key reuse. Java's Arrays.sort is used as + * reference, which is expected to be faster but it can only sort a single array. Sorter can be + * used to sort parallel arrays. + * + * Ideally these would be executed one at a time, each in their own JVM, so their listing + * here is mainly to have the code. Running multiple tests within the same JVM session would + * prevent JIT inlining overridden methods and hence hurt the performance. + */ + test("Sorter benchmark for primitive int array") { + val numElements = 25000000 // 25 mil + val rand = new XORShiftRandom(123) + + val ints = Array.fill(numElements)(rand.nextInt()) + val intObjects = { + val data = new Array[JInteger](numElements) + var i = 0 + while (i < numElements) { + data(i) = new JInteger(ints(i)) + i += 1 + } + data } - // Test non-primitive sort on float array - runExperiment("Java Arrays.sort()") { - Arrays.sort(keys, new Comparator[JFloat] { - override def compare(x: JFloat, y: JFloat): Int = Ordering.Float.compare(x, y) - }) + val intObjectArray = new Array[JInteger](numElements) + val prepareIntObjectArray = () => { + System.arraycopy(intObjects, 0, intObjectArray, 0, numElements) } - // Test primitive sort on float array - val primitiveKeys = Array.tabulate[Float](numElements) { i => rand.nextFloat() } - runExperiment("Java Arrays.sort() on primitive keys") { - Arrays.sort(primitiveKeys) + runExperiment("Java Arrays.sort() on non-primitive int array")({ + Arrays.sort(intObjectArray, new Comparator[JInteger] { + override def compare(x: JInteger, y: JInteger): Int = x.compareTo(y) + }) + }, prepareIntObjectArray) + + val intPrimitiveArray = new Array[Int](numElements) + val prepareIntPrimitiveArray = () => { + System.arraycopy(ints, 0, intPrimitiveArray, 0, numElements) } - } -} + runExperiment("Java Arrays.sort() on primitive int array")({ + Arrays.sort(intPrimitiveArray) + }, prepareIntPrimitiveArray) -/** Format to sort a simple Array[Int]. Could be easily generified and specialized. */ -class IntArraySortDataFormat extends SortDataFormat[Int, Array[Int]] { - override protected def getKey(data: Array[Int], pos: Int): Int = { - data(pos) + val sorterWithoutKeyReuse = new Sorter(new IntArraySortDataFormat) + runExperiment("Sorter without key reuse on primitive int array")({ + sorterWithoutKeyReuse.sort(intPrimitiveArray, 0, numElements, Ordering[Int]) + }, prepareIntPrimitiveArray) + + val sorterWithKeyReuse = new Sorter(new KeyReuseIntArraySortDataFormat) + runExperiment("Sorter with key reuse on primitive int array")({ + sorterWithKeyReuse.sort(intPrimitiveArray, 0, numElements, Ordering[IntWrapper]) + }, prepareIntPrimitiveArray) } +} - override protected def swap(data: Array[Int], pos0: Int, pos1: Int): Unit = { +abstract class AbstractIntArraySortDataFormat[K] extends SortDataFormat[K, Array[Int]] { + + override def swap(data: Array[Int], pos0: Int, pos1: Int): Unit = { val tmp = data(pos0) data(pos0) = data(pos1) data(pos1) = tmp } - override protected def copyElement(src: Array[Int], srcPos: Int, dst: Array[Int], dstPos: Int) { + override def copyElement(src: Array[Int], srcPos: Int, dst: Array[Int], dstPos: Int) { dst(dstPos) = src(srcPos) } /** Copy a range of elements starting at src(srcPos) to dest, starting at destPos. */ - override protected def copyRange(src: Array[Int], srcPos: Int, - dst: Array[Int], dstPos: Int, length: Int) { + override def copyRange(src: Array[Int], srcPos: Int, dst: Array[Int], dstPos: Int, length: Int) { System.arraycopy(src, srcPos, dst, dstPos, length) } /** Allocates a new structure that can hold up to 'length' elements. */ - override protected def allocate(length: Int): Array[Int] = { + override def allocate(length: Int): Array[Int] = { new Array[Int](length) } } + +/** Format to sort a simple Array[Int]. Could be easily generified and specialized. */ +class IntArraySortDataFormat extends AbstractIntArraySortDataFormat[Int] { + + override protected def getKey(data: Array[Int], pos: Int): Int = { + data(pos) + } +} + +/** Wrapper of Int for key reuse. */ +class IntWrapper(var key: Int = 0) extends Ordered[IntWrapper] { + + override def compare(that: IntWrapper): Int = { + Ordering.Int.compare(key, that.key) + } +} + +/** SortDataFormat for Array[Int] with reused keys. */ +class KeyReuseIntArraySortDataFormat extends AbstractIntArraySortDataFormat[IntWrapper] { + + override def newKey(): IntWrapper = { + new IntWrapper() + } + + override def getKey(data: Array[Int], pos: Int, reuse: IntWrapper): IntWrapper = { + if (reuse == null) { + new IntWrapper(data(pos)) + } else { + reuse.key = data(pos) + reuse + } + } + + override protected def getKey(data: Array[Int], pos: Int): IntWrapper = { + getKey(data, pos, null) + } +} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index c58666af84f24..95152b58e287e 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -53,7 +53,9 @@ object MimaExcludes { "org.apache.spark.scheduler.MapStatus"), // TaskContext was promoted to Abstract class ProblemFilters.exclude[AbstractClassProblem]( - "org.apache.spark.TaskContext") + "org.apache.spark.TaskContext"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.util.collection.SortDataFormat") ) ++ Seq( // Adding new methods to the JavaRDDLike trait: ProblemFilters.exclude[MissingMethodProblem]( From 1536d70331e9a4f5b5ea9dabfd72592ca1fc8e35 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 28 Oct 2014 17:59:10 -0700 Subject: [PATCH 071/115] [SPARK-4008] Fix "kryo with fold" in KryoSerializerSuite `zeroValue` will be serialized by `spark.closure.serializer` but `spark.closure.serializer` only supports the default Java serializer. So it must not be `ClassWithoutNoArgConstructor`, which can not be serialized by the Java serializer. This PR changed `zeroValue` to null and updated the test to make it work correctly. Author: zsxwing Closes #2856 from zsxwing/SPARK-4008 and squashes the following commits: 51da655 [zsxwing] [SPARK-4008] Fix "kryo with fold" in KryoSerializerSuite --- .../spark/serializer/KryoSerializerSuite.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 64ac6d2d920d2..a70f67af2e62e 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -201,12 +201,17 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { assert(control.sum === result) } - // TODO: this still doesn't work - ignore("kryo with fold") { + test("kryo with fold") { val control = 1 :: 2 :: Nil + // zeroValue must not be a ClassWithoutNoArgConstructor instance because it will be + // serialized by spark.closure.serializer but spark.closure.serializer only supports + // the default Java serializer. val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)) - .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x - assert(10 + control.sum === result) + .fold(null)((t1, t2) => { + val t1x = if (t1 == null) 0 else t1.x + new ClassWithoutNoArgConstructor(t1x + t2.x) + }).x + assert(control.sum === result) } test("kryo with nonexistent custom registrator should fail") { From b5e79bf889700159d490cdac1f6322dff424b1d9 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 28 Oct 2014 19:11:57 -0700 Subject: [PATCH 072/115] [SPARK-3904] [SQL] add constant objectinspector support for udfs In HQL, we convert all of the data type into normal `ObjectInspector`s for UDFs, most of cases it works, however, some of the UDF actually requires its children `ObjectInspector` to be the `ConstantObjectInspector`, which will cause exception. e.g. select named_struct("x", "str") from src limit 1; I updated the method `wrap` by adding the one more parameter `ObjectInspector`(to describe what it expects to wrap to, for example: java.lang.Integer or IntWritable). As well as the `unwrap` method by providing the input `ObjectInspector`. Author: Cheng Hao Closes #2762 from chenghao-intel/udf_coi and squashes the following commits: bcacfd7 [Cheng Hao] Shim for both Hive 0.12 & 0.13.1 2416e5d [Cheng Hao] revert to hive 0.12 5793c01 [Cheng Hao] add space before while 4e56e1b [Cheng Hao] style issue 683d3fd [Cheng Hao] Add golden files fe591e4 [Cheng Hao] update HiveGenericUdf for set the ObjectInspector while constructing the DeferredObject f6740fe [Cheng Hao] Support Constant ObjectInspector for Map & List 8814c3a [Cheng Hao] Passing ContantObjectInspector(when necessary) for UDF initializing --- .../execution/HiveCompatibilitySuite.scala | 8 +- .../spark/sql/hive/HiveInspectors.scala | 185 ++++++++++++------ .../apache/spark/sql/hive/TableReader.scala | 2 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 62 +++--- ...nt array-0-761ef205b10ac4a10122c8b4ce10ada | 1 + ...d_struct-0-8f0ea83364b78634fbb3752c5a5c725 | 1 + ..._struct-1-380c9638cc6ea8ea42f187bf0cedf350 | 1 + ..._struct-2-22a79ac608b1249306f82f4bdc669b17 | 0 ..._struct-3-d7e4a555934307155784904ff9df188b | 1 + ...t_array-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...t_array-1-976cd8b6b50a2748bbc768aa5e11cf82 | 1 + ..._array-10-9e047718e5fea6ea79124f1e899f1c13 | 1 + ...t_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 | 4 + ...t_array-3-55c4cdaf8438b06675d60848d68f35de | 0 ..._struct-0-f41043b7d9f14fa5e998c90454c7bdb1 | 1 + ..._struct-1-8ccdb20153debdab789ea8ad0228e2eb | 1 + ..._struct-2-4a62774a6de7571c8d2bcb77da63f8f3 | 0 ..._struct-3-abffdaacb0c7076ab538fbeec072daa2 | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 8 + .../org/apache/spark/sql/hive/Shim12.scala | 57 ++++++ .../org/apache/spark/sql/hive/Shim13.scala | 64 +++++- 21 files changed, 307 insertions(+), 92 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/constant array-0-761ef205b10ac4a10122c8b4ce10ada create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-0-8f0ea83364b78634fbb3752c5a5c725 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-1-380c9638cc6ea8ea42f187bf0cedf350 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-2-22a79ac608b1249306f82f4bdc669b17 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-3-d7e4a555934307155784904ff9df188b create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-10-9e047718e5fea6ea79124f1e899f1c13 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de create mode 100644 sql/hive/src/test/resources/golden/udf_struct-0-f41043b7d9f14fa5e998c90454c7bdb1 create mode 100644 sql/hive/src/test/resources/golden/udf_struct-1-8ccdb20153debdab789ea8ad0228e2eb create mode 100644 sql/hive/src/test/resources/golden/udf_struct-2-4a62774a6de7571c8d2bcb77da63f8f3 create mode 100644 sql/hive/src/test/resources/golden/udf_struct-3-abffdaacb0c7076ab538fbeec072daa2 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 26d9ca05c851b..1a3c24be420e6 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 @@ -233,7 +233,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Sort with Limit clause causes failure. "ctas", - "ctas_hadoop20" + "ctas_hadoop20", + + // timestamp in array, the output format of Hive contains double quotes, while + // Spark SQL doesn't + "udf_sort_array" ) ++ HiveShim.compatibilityBlackList /** @@ -861,6 +865,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_minute", "udf_modulo", "udf_month", + "udf_named_struct", "udf_negative", "udf_not", "udf_notequal", @@ -894,6 +899,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_stddev_pop", "udf_stddev_samp", "udf_string", + "udf_struct", "udf_substring", "udf_subtract", "udf_sum", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index fad7373a2fa39..c6103a124df59 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.hive import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector._ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} @@ -78,44 +80,13 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == classOf[java.lang.Object] => NullType } - /** Converts hive types to native catalyst types. */ - def unwrap(a: Any): Any = a match { - case null => null - case i: hadoopIo.IntWritable => i.get - case t: hadoopIo.Text => t.toString - case l: hadoopIo.LongWritable => l.get - case d: hadoopIo.DoubleWritable => d.get - case d: hiveIo.DoubleWritable => d.get - case s: hiveIo.ShortWritable => s.get - case b: hadoopIo.BooleanWritable => b.get - case b: hiveIo.ByteWritable => b.get - case b: hadoopIo.FloatWritable => b.get - case b: hadoopIo.BytesWritable => { - val bytes = new Array[Byte](b.getLength) - System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) - bytes - } - case d: hiveIo.DateWritable => d.get - case t: hiveIo.TimestampWritable => t.getTimestamp - case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) - case list: java.util.List[_] => list.map(unwrap) - case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap - case array: Array[_] => array.map(unwrap).toSeq - case p: java.lang.Short => p - case p: java.lang.Long => p - case p: java.lang.Float => p - case p: java.lang.Integer => p - case p: java.lang.Double => p - case p: java.lang.Byte => p - case p: java.lang.Boolean => p - case str: String => str - case p: java.math.BigDecimal => p - case p: Array[Byte] => p - case p: java.sql.Date => p - case p: java.sql.Timestamp => p - } - - def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { + /** + * Converts hive types to native catalyst types. + * @param data the data in Hive type + * @param oi the ObjectInspector associated with the Hive Type + * @return convert the data into catalyst type + */ + def unwrap(data: Any, oi: ObjectInspector): Any = oi match { case hvoi: HiveVarcharObjectInspector => if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue case hdoi: HiveDecimalObjectInspector => @@ -123,43 +94,89 @@ private[hive] trait HiveInspectors { case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) case li: ListObjectInspector => Option(li.getList(data)) - .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) + .map(_.map(unwrap(_, li.getListElementObjectInspector)).toSeq) .orNull case mi: MapObjectInspector => Option(mi.getMap(data)).map( _.map { case (k,v) => - (unwrapData(k, mi.getMapKeyObjectInspector), - unwrapData(v, mi.getMapValueObjectInspector)) + (unwrap(k, mi.getMapKeyObjectInspector), + unwrap(v, mi.getMapValueObjectInspector)) }.toMap).orNull case si: StructObjectInspector => val allRefs = si.getAllStructFieldRefs new GenericRow( allRefs.map(r => - unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) + unwrap(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) } - /** Converts native catalyst types to the types expected by Hive */ - def wrap(a: Any): AnyRef = a match { - case s: String => s: java.lang.String - case i: Int => i: java.lang.Integer - case b: Boolean => b: java.lang.Boolean - case f: Float => f: java.lang.Float - case d: Double => d: java.lang.Double - case l: Long => l: java.lang.Long - case l: Short => l: java.lang.Short - case l: Byte => l: java.lang.Byte - case b: BigDecimal => HiveShim.createDecimal(b.underlying()) - case b: Array[Byte] => b - case d: java.sql.Date => d - case t: java.sql.Timestamp => t - case s: Seq[_] => seqAsJavaList(s.map(wrap)) - case m: Map[_,_] => - // Some UDFs seem to assume we pass in a HashMap. - val hashMap = new java.util.HashMap[AnyRef, AnyRef]() - hashMap.putAll(m.map { case (k, v) => wrap(k) -> wrap(v) }) - hashMap - case null => null + /** + * Converts native catalyst types to the types expected by Hive + * @param a the value to be wrapped + * @param oi This ObjectInspector associated with the value returned by this function, and + * the ObjectInspector should also be consistent with those returned from + * toInspector: DataType => ObjectInspector and + * toInspector: Expression => ObjectInspector + */ + def wrap(a: Any, oi: ObjectInspector): AnyRef = if (a == null) { + null + } else { + oi match { + case x: ConstantObjectInspector => x.getWritableConstantValue + case x: PrimitiveObjectInspector => a match { + // TODO what if x.preferWritable() == true? reuse the writable? + case s: String => s: java.lang.String + case i: Int => i: java.lang.Integer + case b: Boolean => b: java.lang.Boolean + case f: Float => f: java.lang.Float + case d: Double => d: java.lang.Double + case l: Long => l: java.lang.Long + case l: Short => l: java.lang.Short + case l: Byte => l: java.lang.Byte + case b: BigDecimal => HiveShim.createDecimal(b.underlying()) + case b: Array[Byte] => b + case d: java.sql.Date => d + case t: java.sql.Timestamp => t + } + case x: StructObjectInspector => + val fieldRefs = x.getAllStructFieldRefs + val row = a.asInstanceOf[Seq[_]] + val result = new java.util.ArrayList[AnyRef](fieldRefs.length) + var i = 0 + while (i < fieldRefs.length) { + result.add(wrap(row(i), fieldRefs.get(i).getFieldObjectInspector)) + i += 1 + } + + result + case x: ListObjectInspector => + val list = new java.util.ArrayList[Object] + a.asInstanceOf[Seq[_]].foreach { + v => list.add(wrap(v, x.getListElementObjectInspector)) + } + list + case x: MapObjectInspector => + // Some UDFs seem to assume we pass in a HashMap. + val hashMap = new java.util.HashMap[AnyRef, AnyRef]() + hashMap.putAll(a.asInstanceOf[Map[_, _]].map { + case (k, v) => + wrap(k, x.getMapKeyObjectInspector) -> wrap(v, x.getMapValueObjectInspector) + }) + + hashMap + } + } + + def wrap( + row: Seq[Any], + inspectors: Seq[ObjectInspector], + cache: Array[AnyRef]): Array[AnyRef] = { + var i = 0 + while (i < inspectors.length) { + cache(i) = wrap(row(i), inspectors(i)) + i += 1 + } + cache } def toInspector(dataType: DataType): ObjectInspector = dataType match { @@ -186,6 +203,48 @@ private[hive] trait HiveInspectors { fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) } + def toInspector(expr: Expression): ObjectInspector = expr match { + case Literal(value: String, StringType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Int, IntegerType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Double, DoubleType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Boolean, BooleanType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Long, LongType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Float, FloatType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Short, ShortType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Byte, ByteType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Array[Byte], BinaryType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: java.sql.Date, DateType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: java.sql.Timestamp, TimestampType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: BigDecimal, DecimalType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(_, NullType) => + HiveShim.getPrimitiveNullWritableConstantObjectInspector + case Literal(value: Seq[_], ArrayType(dt, _)) => + val listObjectInspector = toInspector(dt) + val list = new java.util.ArrayList[Object]() + value.foreach(v => list.add(wrap(v, listObjectInspector))) + ObjectInspectorFactory.getStandardConstantListObjectInspector(listObjectInspector, list) + case Literal(map: Map[_, _], MapType(keyType, valueType, _)) => + val value = new java.util.HashMap[Object, Object]() + val keyOI = toInspector(keyType) + val valueOI = toInspector(valueType) + map.foreach (entry => value.put(wrap(entry._1, keyOI), wrap(entry._2, valueOI))) + ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, value) + case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].") + case _ => toInspector(expr.dataType) + } + def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { case s: StructObjectInspector => StructType(s.getAllStructFieldRefs.map(f => { 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 9ff7ab5a124c1..e49f0957d188a 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 @@ -291,7 +291,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { case oi: DoubleObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) case oi => - (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrapData(value, oi) + (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrap(value, oi) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 683c820dec305..aff4ddce92272 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -21,7 +21,9 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} @@ -97,7 +99,7 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) @transient - protected lazy val arguments = children.map(c => toInspector(c.dataType)).toArray + protected lazy val arguments = children.map(toInspector).toArray @transient protected lazy val isUDFDeterministic = { @@ -116,12 +118,19 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ @transient lazy val dataType = javaClassToDataType(method.getReturnType) + @transient + lazy val returnInspector = ObjectInspectorFactory.getReflectionObjectInspector( + method.getGenericReturnType(), ObjectInspectorOptions.JAVA) + + @transient + protected lazy val cached = new Array[AnyRef](children.length) + // TODO: Finish input output types. override def eval(input: Row): Any = { - val evaluatedChildren = children.map(c => wrap(c.eval(input))) - - unwrap(FunctionRegistry.invoke(method, function, conversionHelper - .convertIfNecessary(evaluatedChildren: _*): _*)) + unwrap( + FunctionRegistry.invoke(method, function, conversionHelper + .convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _*): _*), + returnInspector) } } @@ -133,7 +142,7 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq type UDFType = GenericUDF @transient - protected lazy val argumentInspectors = children.map(_.dataType).map(toInspector) + protected lazy val argumentInspectors = children.map(toInspector) @transient protected lazy val returnInspector = function.initialize(argumentInspectors.toArray) @@ -148,18 +157,18 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable) } - protected lazy val deferedObjects = Array.fill[DeferredObject](children.length)({ - new DeferredObjectAdapter - }) + @transient + protected lazy val deferedObjects = + argumentInspectors.map(new DeferredObjectAdapter(_)).toArray[DeferredObject] // Adapter from Catalyst ExpressionResult to Hive DeferredObject - class DeferredObjectAdapter extends DeferredObject { + class DeferredObjectAdapter(oi: ObjectInspector) extends DeferredObject { private var func: () => Any = _ def set(func: () => Any) { this.func = func } override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(func()) + override def get(): AnyRef = wrap(func(), oi) } lazy val dataType: DataType = inspectorToDataType(returnInspector) @@ -169,10 +178,13 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq var i = 0 while (i < children.length) { val idx = i - deferedObjects(i).asInstanceOf[DeferredObjectAdapter].set(() => {children(idx).eval(input)}) + deferedObjects(i).asInstanceOf[DeferredObjectAdapter].set( + () => { + children(idx).eval(input) + }) i += 1 } - unwrap(function.evaluate(deferedObjects)) + unwrap(function.evaluate(deferedObjects), returnInspector) } } @@ -260,12 +272,14 @@ private[hive] case class HiveGenericUdtf( protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) @transient - protected lazy val outputInspectors = { - val structInspector = function.initialize(inputInspectors.toArray) - structInspector.getAllStructFieldRefs.map(_.getFieldObjectInspector) - } + protected lazy val outputInspector = function.initialize(inputInspectors.toArray) - protected lazy val outputDataTypes = outputInspectors.map(inspectorToDataType) + @transient + protected lazy val udtInput = new Array[AnyRef](children.length) + + protected lazy val outputDataTypes = outputInspector.getAllStructFieldRefs.map { + field => inspectorToDataType(field.getFieldObjectInspector) + } override protected def makeOutput() = { // Use column names when given, otherwise c_1, c_2, ... c_n. @@ -283,14 +297,12 @@ private[hive] case class HiveGenericUdtf( } override def eval(input: Row): TraversableOnce[Row] = { - outputInspectors // Make sure initialized. + outputInspector // Make sure initialized. val inputProjection = new InterpretedProjection(children) val collector = new UDTFCollector function.setCollector(collector) - - val udtInput = inputProjection(input).map(wrap).toArray - function.process(udtInput) + function.process(wrap(inputProjection(input), inputInspectors, udtInput)) collector.collectRows() } @@ -301,7 +313,7 @@ private[hive] case class HiveGenericUdtf( // We need to clone the input here because implementations of // GenericUDTF reuse the same object. Luckily they are always an array, so // it is easy to clone. - collected += new GenericRow(input.asInstanceOf[Array[_]].map(unwrap)) + collected += unwrap(input, outputInspector).asInstanceOf[Row] } def collectRows() = { @@ -342,7 +354,7 @@ private[hive] case class HiveUdafFunction( private val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - override def eval(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) + override def eval(input: Row): Any = unwrap(function.evaluate(buffer), returnInspector) @transient val inputProjection = new InterpretedProjection(exprs) diff --git a/sql/hive/src/test/resources/golden/constant array-0-761ef205b10ac4a10122c8b4ce10ada b/sql/hive/src/test/resources/golden/constant array-0-761ef205b10ac4a10122c8b4ce10ada new file mode 100644 index 0000000000000..94f18d09863a7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/constant array-0-761ef205b10ac4a10122c8b4ce10ada @@ -0,0 +1 @@ +["enterprise databases","hadoop distributed file system","hadoop map-reduce"] diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-0-8f0ea83364b78634fbb3752c5a5c725 b/sql/hive/src/test/resources/golden/udf_named_struct-0-8f0ea83364b78634fbb3752c5a5c725 new file mode 100644 index 0000000000000..9bff96e7fa20e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-0-8f0ea83364b78634fbb3752c5a5c725 @@ -0,0 +1 @@ +named_struct(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-1-380c9638cc6ea8ea42f187bf0cedf350 b/sql/hive/src/test/resources/golden/udf_named_struct-1-380c9638cc6ea8ea42f187bf0cedf350 new file mode 100644 index 0000000000000..9bff96e7fa20e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-1-380c9638cc6ea8ea42f187bf0cedf350 @@ -0,0 +1 @@ +named_struct(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-2-22a79ac608b1249306f82f4bdc669b17 b/sql/hive/src/test/resources/golden/udf_named_struct-2-22a79ac608b1249306f82f4bdc669b17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-3-d7e4a555934307155784904ff9df188b b/sql/hive/src/test/resources/golden/udf_named_struct-3-d7e4a555934307155784904ff9df188b new file mode 100644 index 0000000000000..de25f51b5b56d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-3-d7e4a555934307155784904ff9df188b @@ -0,0 +1 @@ +{"foo":1,"bar":2} 1 diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 b/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 new file mode 100644 index 0000000000000..d514df4191b89 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 @@ -0,0 +1 @@ +sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-10-9e047718e5fea6ea79124f1e899f1c13 b/sql/hive/src/test/resources/golden/udf_sort_array-10-9e047718e5fea6ea79124f1e899f1c13 new file mode 100644 index 0000000000000..9d33cd51fef04 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-10-9e047718e5fea6ea79124f1e899f1c13 @@ -0,0 +1 @@ +[1,2,3,4,5] [1,2,7,8,9] [4,8,16,32,64] [1,100,246,357,1000] [false,true] [1.414,1.618,2.718,3.141] [1.41421,1.61803,2.71828,3.14159] ["","aramis","athos","portos"] ["1970-01-05 13:51:04.042","1970-01-07 00:54:54.442","1970-01-16 12:50:35.242"] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 b/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 new file mode 100644 index 0000000000000..43e36513de881 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 @@ -0,0 +1,4 @@ +sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. +Example: + > SELECT sort_array(array('b', 'd', 'c', 'a')) FROM src LIMIT 1; + 'a', 'b', 'c', 'd' diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de b/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_struct-0-f41043b7d9f14fa5e998c90454c7bdb1 b/sql/hive/src/test/resources/golden/udf_struct-0-f41043b7d9f14fa5e998c90454c7bdb1 new file mode 100644 index 0000000000000..062cb1bc683b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-0-f41043b7d9f14fa5e998c90454c7bdb1 @@ -0,0 +1 @@ +struct(col1, col2, col3, ...) - Creates a struct with the given field values diff --git a/sql/hive/src/test/resources/golden/udf_struct-1-8ccdb20153debdab789ea8ad0228e2eb b/sql/hive/src/test/resources/golden/udf_struct-1-8ccdb20153debdab789ea8ad0228e2eb new file mode 100644 index 0000000000000..062cb1bc683b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-1-8ccdb20153debdab789ea8ad0228e2eb @@ -0,0 +1 @@ +struct(col1, col2, col3, ...) - Creates a struct with the given field values diff --git a/sql/hive/src/test/resources/golden/udf_struct-2-4a62774a6de7571c8d2bcb77da63f8f3 b/sql/hive/src/test/resources/golden/udf_struct-2-4a62774a6de7571c8d2bcb77da63f8f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_struct-3-abffdaacb0c7076ab538fbeec072daa2 b/sql/hive/src/test/resources/golden/udf_struct-3-abffdaacb0c7076ab538fbeec072daa2 new file mode 100644 index 0000000000000..ff1a28fa47f18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-3-abffdaacb0c7076ab538fbeec072daa2 @@ -0,0 +1 @@ +{"col1":1} {"col1":1,"col2":"a"} 1 a 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 322a25bb20837..ffe1f0b90fcd0 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 @@ -34,6 +34,14 @@ case class TestData(a: Int, b: String) * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("constant array", + """ + |SELECT sort_array( + | sort_array( + | array("hadoop distributed file system", + | "enterprise databases", "hadoop map-reduce"))) + |FROM src LIMIT 1; + """.stripMargin) createQueryTest("count distinct 0 values", """ diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 8cb81db8a9724..afc252ac27987 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -29,7 +29,11 @@ import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.stats.StatsSetupConst +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} +import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} import org.apache.hadoop.mapred.InputFormat import scala.collection.JavaConversions._ @@ -50,6 +54,59 @@ private[hive] object HiveShim { new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) } + def getPrimitiveWritableConstantObjectInspector(value: String): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.STRING, new hadoopIo.Text(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Int): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.INT, new hadoopIo.IntWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Double): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.DOUBLE, new hiveIo.DoubleWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Boolean): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.BOOLEAN, new hadoopIo.BooleanWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Long): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.LONG, new hadoopIo.LongWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Float): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.FLOAT, new hadoopIo.FloatWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Short): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.SHORT, new hiveIo.ShortWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Byte): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.BYTE, new hiveIo.ByteWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Array[Byte]): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.BINARY, new hadoopIo.BytesWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: java.sql.Date): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.DATE, new hiveIo.DateWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: java.sql.Timestamp): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.TIMESTAMP, new hiveIo.TimestampWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: BigDecimal): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.DECIMAL, + new hiveIo.HiveDecimalWritable(HiveShim.createDecimal(value.underlying()))) + + def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.VOID, null) + def createDriverResultsArray = new JArrayList[String] def processResults(results: JArrayList[String]) = results diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index b9a742cc6ef67..42cd65b2518c9 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -21,6 +21,7 @@ import java.util.{ArrayList => JArrayList} import java.util.Properties import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.common.`type`.{HiveDecimal} import org.apache.hadoop.hive.conf.HiveConf @@ -28,10 +29,16 @@ import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} -import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.Logging +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} +import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} +import org.apache.spark.Logging + import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -54,6 +61,59 @@ private[hive] object HiveShim { new TableDesc(inputFormatClass, outputFormatClass, properties) } + def getPrimitiveWritableConstantObjectInspector(value: String): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.stringTypeInfo, new hadoopIo.Text(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Int): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.intTypeInfo, new hadoopIo.IntWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Double): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.doubleTypeInfo, new hiveIo.DoubleWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Boolean): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.booleanTypeInfo, new hadoopIo.BooleanWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Long): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.longTypeInfo, new hadoopIo.LongWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Float): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.floatTypeInfo, new hadoopIo.FloatWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Short): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.shortTypeInfo, new hiveIo.ShortWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Byte): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.byteTypeInfo, new hiveIo.ByteWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Array[Byte]): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.binaryTypeInfo, new hadoopIo.BytesWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: java.sql.Date): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.dateTypeInfo, new hiveIo.DateWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: java.sql.Timestamp): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.timestampTypeInfo, new hiveIo.TimestampWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: BigDecimal): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.decimalTypeInfo, + new hiveIo.HiveDecimalWritable(HiveShim.createDecimal(value.underlying()))) + + def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.voidTypeInfo, null) + def createDriverResultsArray = new JArrayList[Object] def processResults(results: JArrayList[Object]) = { From 8c0bfd08fc19fa5de7d77bf8306d19834f907ec0 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 28 Oct 2014 19:38:16 -0700 Subject: [PATCH 073/115] [SPARK-4133] [SQL] [PySpark] type conversionfor python udf Call Python UDF on ArrayType/MapType/PrimitiveType, the returnType can also be ArrayType/MapType/PrimitiveType. For StructType, it will act as tuple (without attributes). If returnType is StructType, it also should be tuple. Author: Davies Liu Closes #2973 from davies/udf_array and squashes the following commits: 306956e [Davies Liu] Merge branch 'master' of github.com:apache/spark into udf_array 2c00e43 [Davies Liu] fix merge 11395fa [Davies Liu] Merge branch 'master' of github.com:apache/spark into udf_array 9df50a2 [Davies Liu] address comments 79afb4e [Davies Liu] type conversionfor python udf --- python/pyspark/tests.py | 16 +++- .../org/apache/spark/sql/SQLContext.scala | 43 +-------- .../org/apache/spark/sql/SchemaRDD.scala | 42 +-------- .../spark/sql/execution/pythonUdfs.scala | 91 +++++++++++++++++-- 4 files changed, 102 insertions(+), 90 deletions(-) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 047d85783089f..37a128907b3a7 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -49,7 +49,7 @@ from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter -from pyspark.sql import SQLContext, IntegerType, Row +from pyspark.sql import SQLContext, IntegerType, Row, ArrayType from pyspark import shuffle _have_scipy = False @@ -690,10 +690,20 @@ def test_udf(self): self.assertEqual(row[0], 5) def test_udf2(self): - self.sqlCtx.registerFunction("strlen", lambda string: len(string)) + self.sqlCtx.registerFunction("strlen", lambda string: len(string), IntegerType()) self.sqlCtx.inferSchema(self.sc.parallelize([Row(a="test")])).registerTempTable("test") [res] = self.sqlCtx.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() - self.assertEqual(u"4", res[0]) + self.assertEqual(4, res[0]) + + def test_udf_with_array_type(self): + d = [Row(l=range(3), d={"key": range(5)})] + rdd = self.sc.parallelize(d) + srdd = self.sqlCtx.inferSchema(rdd).registerTempTable("test") + self.sqlCtx.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) + self.sqlCtx.registerFunction("maplen", lambda d: len(d), IntegerType()) + [(l1, l2)] = self.sqlCtx.sql("select copylist(l), maplen(d) from test").collect() + self.assertEqual(range(3), l1) + self.assertEqual(1, l2) def test_broadcast_in_udf(self): bar = {"a": "aa", "b": "bb", "c": "abc"} 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 ca8706ee68697..a41a500c9a5d0 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 @@ -438,7 +438,6 @@ class SQLContext(@transient val sparkContext: SparkContext) private[sql] def applySchemaToPythonRDD( rdd: RDD[Array[Any]], schema: StructType): SchemaRDD = { - import scala.collection.JavaConversions._ def needsConversion(dataType: DataType): Boolean = dataType match { case ByteType => true @@ -452,49 +451,9 @@ class SQLContext(@transient val sparkContext: SparkContext) case other => false } - // Converts value to the type specified by the data type. - // Because Python does not have data types for DateType, TimestampType, FloatType, ShortType, - // and ByteType, we need to explicitly convert values in columns of these data types to the - // desired JVM data types. - def convert(obj: Any, dataType: DataType): Any = (obj, dataType) match { - // TODO: We should check nullable - case (null, _) => null - - case (c: java.util.List[_], ArrayType(elementType, _)) => - c.map { e => convert(e, elementType)}: Seq[Any] - - case (c, ArrayType(elementType, _)) if c.getClass.isArray => - c.asInstanceOf[Array[_]].map(e => convert(e, elementType)): Seq[Any] - - case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => c.map { - case (key, value) => (convert(key, keyType), convert(value, valueType)) - }.toMap - - case (c, StructType(fields)) if c.getClass.isArray => - new GenericRow(c.asInstanceOf[Array[_]].zip(fields).map { - case (e, f) => convert(e, f.dataType) - }): Row - - case (c: java.util.Calendar, DateType) => - new java.sql.Date(c.getTime().getTime()) - - case (c: java.util.Calendar, TimestampType) => - new java.sql.Timestamp(c.getTime().getTime()) - - case (c: Int, ByteType) => c.toByte - case (c: Long, ByteType) => c.toByte - case (c: Int, ShortType) => c.toShort - case (c: Long, ShortType) => c.toShort - case (c: Long, IntegerType) => c.toInt - case (c: Double, FloatType) => c.toFloat - case (c, StringType) if !c.isInstanceOf[String] => c.toString - - case (c, _) => c - } - val convertedRdd = if (schema.fields.exists(f => needsConversion(f.dataType))) { rdd.map(m => m.zip(schema.fields).map { - case (value, field) => convert(value, field.dataType) + case (value, field) => EvaluatePython.fromJava(value, field.dataType) }) } else { rdd diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 948122d42f0e1..8b96df10963b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.execution.LogicalRDD +import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} import org.apache.spark.api.java.JavaRDD /** @@ -377,47 +377,15 @@ class SchemaRDD( */ def toJavaSchemaRDD: JavaSchemaRDD = new JavaSchemaRDD(sqlContext, logicalPlan) - /** - * Helper for converting a Row to a simple Array suitable for pyspark serialization. - */ - private def rowToJArray(row: Row, structType: StructType): Array[Any] = { - import scala.collection.Map - - def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { - case (null, _) => null - - case (obj: Row, struct: StructType) => rowToJArray(obj, struct) - - case (seq: Seq[Any], array: ArrayType) => - seq.map(x => toJava(x, array.elementType)).asJava - case (list: JList[_], array: ArrayType) => - list.map(x => toJava(x, array.elementType)).asJava - case (arr, array: ArrayType) if arr.getClass.isArray => - arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) - - case (obj: Map[_, _], mt: MapType) => obj.map { - case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type - }.asJava - - // Pyrolite can handle Timestamp - case (other, _) => other - } - - val fields = structType.fields.map(field => field.dataType) - row.zip(fields).map { - case (obj, dataType) => toJava(obj, dataType) - }.toArray - } - /** * Converts a JavaRDD to a PythonRDD. It is used by pyspark. */ private[sql] def javaToPython: JavaRDD[Array[Byte]] = { - val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) + val fieldTypes = schema.fields.map(_.dataType) this.mapPartitions { iter => val pickle = new Pickler iter.map { row => - rowToJArray(row, rowSchema) + EvaluatePython.rowToArray(row, fieldTypes) }.grouped(100).map(batched => pickle.dumps(batched.toArray)) } } @@ -427,10 +395,10 @@ class SchemaRDD( * format as javaToPython. It is used by pyspark. */ private[sql] def collectToPython: JList[Array[Byte]] = { - val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) + val fieldTypes = schema.fields.map(_.dataType) val pickle = new Pickler new java.util.ArrayList(collect().map { row => - rowToJArray(row, rowSchema) + EvaluatePython.rowToArray(row, fieldTypes) }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index be729e5d244b0..a1961bba1899e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -19,11 +19,14 @@ package org.apache.spark.sql.execution import java.util.{List => JList, Map => JMap} +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonRDD import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -31,8 +34,6 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.types._ import org.apache.spark.{Accumulator, Logging => SparkLogging} -import scala.collection.JavaConversions._ - /** * A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]]. */ @@ -108,6 +109,80 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { object EvaluatePython { def apply(udf: PythonUDF, child: LogicalPlan) = new EvaluatePython(udf, child, AttributeReference("pythonUDF", udf.dataType)()) + + /** + * Helper for converting a Scala object to a java suitable for pyspark serialization. + */ + def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { + case (null, _) => null + + case (row: Row, struct: StructType) => + val fields = struct.fields.map(field => field.dataType) + row.zip(fields).map { + case (obj, dataType) => toJava(obj, dataType) + }.toArray + + case (seq: Seq[Any], array: ArrayType) => + seq.map(x => toJava(x, array.elementType)).asJava + case (list: JList[_], array: ArrayType) => + list.map(x => toJava(x, array.elementType)).asJava + case (arr, array: ArrayType) if arr.getClass.isArray => + arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) + + case (obj: Map[_, _], mt: MapType) => obj.map { + case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type + }.asJava + + // Pyrolite can handle Timestamp + case (other, _) => other + } + + /** + * Convert Row into Java Array (for pickled into Python) + */ + def rowToArray(row: Row, fields: Seq[DataType]): Array[Any] = { + row.zip(fields).map {case (obj, dt) => toJava(obj, dt)}.toArray + } + + // Converts value to the type specified by the data type. + // Because Python does not have data types for TimestampType, FloatType, ShortType, and + // ByteType, we need to explicitly convert values in columns of these data types to the desired + // JVM data types. + def fromJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { + // TODO: We should check nullable + case (null, _) => null + + case (c: java.util.List[_], ArrayType(elementType, _)) => + c.map { e => fromJava(e, elementType)}: Seq[Any] + + case (c, ArrayType(elementType, _)) if c.getClass.isArray => + c.asInstanceOf[Array[_]].map(e => fromJava(e, elementType)): Seq[Any] + + case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => c.map { + case (key, value) => (fromJava(key, keyType), fromJava(value, valueType)) + }.toMap + + case (c, StructType(fields)) if c.getClass.isArray => + new GenericRow(c.asInstanceOf[Array[_]].zip(fields).map { + case (e, f) => fromJava(e, f.dataType) + }): Row + + case (c: java.util.Calendar, DateType) => + new java.sql.Date(c.getTime().getTime()) + + case (c: java.util.Calendar, TimestampType) => + new java.sql.Timestamp(c.getTime().getTime()) + + case (c: Int, ByteType) => c.toByte + case (c: Long, ByteType) => c.toByte + case (c: Int, ShortType) => c.toShort + case (c: Long, ShortType) => c.toShort + case (c: Long, IntegerType) => c.toInt + case (c: Double, FloatType) => c.toFloat + case (c, StringType) if !c.isInstanceOf[String] => c.toString + + case (c, _) => c + } } /** @@ -141,8 +216,11 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: val parent = childResults.mapPartitions { iter => val pickle = new Pickler val currentRow = newMutableProjection(udf.children, child.output)() + val fields = udf.children.map(_.dataType) iter.grouped(1000).map { inputRows => - val toBePickled = inputRows.map(currentRow(_).toArray).toArray + val toBePickled = inputRows.map { row => + EvaluatePython.rowToArray(currentRow(row), fields) + }.toArray pickle.dumps(toBePickled) } } @@ -165,10 +243,7 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: }.mapPartitions { iter => val row = new GenericMutableRow(1) iter.map { result => - row(0) = udf.dataType match { - case StringType => result.toString - case other => result - } + row(0) = EvaluatePython.fromJava(result, udf.dataType) row: Row } } From 1559495dd961d299299a27aae2cb940e8c6697c5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 28 Oct 2014 22:00:10 -0700 Subject: [PATCH 074/115] [FIX] disable benchmark code I forgot to disable the benchmark code in #2937, which increased the Jenkins build time by couple minutes. aarondav Author: Xiangrui Meng Closes #2990 from mengxr/disable-benchmark and squashes the following commits: c58f070 [Xiangrui Meng] disable benchmark code --- .../scala/org/apache/spark/util/collection/SorterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index 066d47c46a0d2..0cb1ed7397655 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -157,7 +157,7 @@ class SorterSuite extends FunSuite { * here is mainly to have the code. Running multiple tests within the same JVM session would * prevent JIT inlining overridden methods and hence hurt the performance. */ - test("Sorter benchmark for primitive int array") { + ignore("Sorter benchmark for primitive int array") { val numElements = 25000000 // 25 mil val rand = new XORShiftRandom(123) From 51ce997355465fc5c29d0e49b92f9bae0bab90ed Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Wed, 29 Oct 2014 10:14:53 -0700 Subject: [PATCH 075/115] [SPARK-4129][MLlib] Performance tuning in MultivariateOnlineSummarizer In MultivariateOnlineSummarizer, breeze's activeIterator is used to loop through the nonZero elements in the vector. However, activeIterator doesn't perform well due to lots of overhead. In this PR, native while loop is used for both DenseVector and SparseVector. The benchmark result with 20 executors using mnist8m dataset: Before: DenseVector: 48.2 seconds SparseVector: 16.3 seconds After: DenseVector: 17.8 seconds SparseVector: 11.2 seconds Since MultivariateOnlineSummarizer is used in several places, the overall performance gain in mllib library will be significant with this PR. Author: DB Tsai Closes #2992 from dbtsai/SPARK-4129 and squashes the following commits: b99db6c [DB Tsai] fixed java.lang.ArrayIndexOutOfBoundsException 2b5e882 [DB Tsai] small refactoring ebe3e74 [DB Tsai] First commit --- .../stat/MultivariateOnlineSummarizer.scala | 25 ++++++++++++++++--- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 3025d4837cab4..fab7c4405c65d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.stat import breeze.linalg.{DenseVector => BDV} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector} /** * :: DeveloperApi :: @@ -72,9 +72,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S require(n == sample.size, s"Dimensions mismatch when adding new sample." + s" Expecting $n but got ${sample.size}.") - sample.toBreeze.activeIterator.foreach { - case (_, 0.0) => // Skip explicit zero elements. - case (i, value) => + @inline def update(i: Int, value: Double) = { + if (value != 0.0) { if (currMax(i) < value) { currMax(i) = value } @@ -89,6 +88,24 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currL1(i) += math.abs(value) nnz(i) += 1.0 + } + } + + sample match { + case dv: DenseVector => { + var j = 0 + while (j < dv.size) { + update(j, dv.values(j)) + j += 1 + } + } + case sv: SparseVector => + var j = 0 + while (j < sv.indices.size) { + update(sv.indices(j), sv.values(j)) + j += 1 + } + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } totalCnt += 1 From dff015533dd7b01b5e392f1ac5f3837e0a65f3f4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 29 Oct 2014 11:27:07 -0700 Subject: [PATCH 076/115] [SPARK-3453] Netty-based BlockTransferService, extracted from Spark core This PR encapsulates #2330, which is itself a continuation of #2240. The first goal of this PR is to provide an alternate, simpler implementation of the ConnectionManager which is based on Netty. In addition to this goal, however, we want to resolve [SPARK-3796](https://issues.apache.org/jira/browse/SPARK-3796), which calls for a standalone shuffle service which can be integrated into the YARN NodeManager, Standalone Worker, or on its own. This PR makes the first step in this direction by ensuring that the actual Netty service is as small as possible and extracted from Spark core. Given this, we should be able to construct this standalone jar which can be included in other JVMs without incurring significant dependency or runtime issues. The actual work to ensure that such a standalone shuffle service would work in Spark will be left for a future PR, however. In order to minimize dependencies and allow for the service to be long-running (possibly much longer-running than Spark, and possibly having to support multiple version of Spark simultaneously), the entire service has been ported to Java, where we have full control over the binary compatibility of the components and do not depend on the Scala runtime or version. These issues: have been addressed by folding in #2330: SPARK-3453: Refactor Netty module to use BlockTransferService interface SPARK-3018: Release all buffers upon task completion/failure SPARK-3002: Create a connection pool and reuse clients across different threads SPARK-3017: Integration tests and unit tests for connection failures SPARK-3049: Make sure client doesn't block when server/connection has error(s) SPARK-3502: SO_RCVBUF and SO_SNDBUF should be bootstrap childOption, not option SPARK-3503: Disable thread local cache in PooledByteBufAllocator TODO before mergeable: - [x] Implement uploadBlock() - [x] Unit tests for RPC side of code - [x] Performance testing (see comments [here](https://github.com/apache/spark/pull/2753#issuecomment-59475022)) - [x] Turn OFF by default (currently on for unit testing) Author: Reynold Xin Author: Aaron Davidson Author: cocoatomo Author: Patrick Wendell Author: Prashant Sharma Author: Davies Liu Author: Anand Avati Closes #2753 from aarondav/netty and squashes the following commits: cadfd28 [Aaron Davidson] Turn netty off by default d7be11b [Aaron Davidson] Turn netty on by default 4a204b8 [Aaron Davidson] Fail block fetches if client connection fails 2b0d1c0 [Aaron Davidson] 100ch 0c5bca2 [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty 14e37f7 [Aaron Davidson] Address Reynold's comments 8dfcceb [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty 322dfc1 [Aaron Davidson] Address Reynold's comments, including major rename e5675a4 [Aaron Davidson] Fail outstanding RPCs as well ccd4959 [Aaron Davidson] Don't throw exception if client immediately fails 9da0bc1 [Aaron Davidson] Add RPC unit tests d236dfd [Aaron Davidson] Remove no-op serializer :) 7b7a26c [Aaron Davidson] Fix Nio compile issue dd420fd [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty-test 939f276 [Aaron Davidson] Attempt to make comm. bidirectional aa58f67 [cocoatomo] [SPARK-3909][PySpark][Doc] A corrupted format in Sphinx documents and building warnings 8dc1ded [cocoatomo] [SPARK-3867][PySpark] ./python/run-tests failed when it run with Python 2.6 and unittest2 is not installed 5b5dbe6 [Prashant Sharma] [SPARK-2924] Required by scala 2.11, only one fun/ctor amongst overriden alternatives, can have default argument(s). 2c5d9dc [Patrick Wendell] HOTFIX: Fix build issue with Akka 2.3.4 upgrade. 020691e [Davies Liu] [SPARK-3886] [PySpark] use AutoBatchedSerializer by default ae4083a [Anand Avati] [SPARK-2805] Upgrade Akka to 2.3.4 29c6dcf [Aaron Davidson] [SPARK-3453] Netty-based BlockTransferService, extracted from Spark core f7e7568 [Reynold Xin] Fixed spark.shuffle.io.receiveBuffer setting. 5d98ce3 [Reynold Xin] Flip buffer. f6c220d [Reynold Xin] Merge with latest master. 407e59a [Reynold Xin] Fix style violation. a0518c7 [Reynold Xin] Implemented block uploads. 4b18db2 [Reynold Xin] Copy the buffer in fetchBlockSync. bec4ea2 [Reynold Xin] Removed OIO and added num threads settings. 1bdd7ee [Reynold Xin] Fixed tests. d68f328 [Reynold Xin] Logging close() in case close() fails. f63fb4c [Reynold Xin] Add more debug message. 6afc435 [Reynold Xin] Added logging. c066309 [Reynold Xin] Implement java.io.Closeable interface. 519d64d [Reynold Xin] Mark private package visibility and MimaExcludes. f0a16e9 [Reynold Xin] Fixed test hanging. 14323a5 [Reynold Xin] Removed BlockManager.getLocalShuffleFromDisk. b2f3281 [Reynold Xin] Added connection pooling. d23ed7b [Reynold Xin] Incorporated feedback from Norman: - use same pool for boss and worker - remove ioratio - disable caching of byte buf allocator - childoption sendbuf/receivebuf - fire exception through pipeline 9e0cb87 [Reynold Xin] Fixed BlockClientHandlerSuite 5cd33d7 [Reynold Xin] Fixed style violation. cb589ec [Reynold Xin] Added more test cases covering cleanup when fault happens in ShuffleBlockFetcherIteratorSuite 1be4e8e [Reynold Xin] Shorten NioManagedBuffer and NettyManagedBuffer class names. 108c9ed [Reynold Xin] Forgot to add TestSerializer to the commit list. b5c8d1f [Reynold Xin] Fixed ShuffleBlockFetcherIteratorSuite. 064747b [Reynold Xin] Reference count buffers and clean them up properly. 2b44cf1 [Reynold Xin] Added more documentation. 1760d32 [Reynold Xin] Use Epoll.isAvailable in BlockServer as well. 165eab1 [Reynold Xin] [SPARK-3453] Refactor Netty module to use BlockTransferService. --- core/pom.xml | 5 + .../scala/org/apache/spark/SparkEnv.scala | 9 +- .../spark/network/BlockDataManager.scala | 14 +- .../spark/network/BlockFetchingListener.scala | 11 +- .../spark/network/BlockTransferService.scala | 28 +- .../apache/spark/network/ManagedBuffer.scala | 166 ----------- .../network/netty/NettyBlockFetcher.scala | 95 +++++++ .../network/netty/NettyBlockRpcServer.scala | 76 +++++ .../netty/NettyBlockTransferService.scala | 111 ++++++++ .../spark/network/netty/NettyConfig.scala | 59 ---- .../netty/client/BlockFetchingClient.scala | 132 --------- .../client/BlockFetchingClientFactory.scala | 99 ------- .../client/BlockFetchingClientHandler.scala | 104 ------- .../netty/client/LazyInitIterator.scala | 44 --- .../netty/client/ReferenceCountedBuffer.scala | 47 ---- .../netty/server/BlockHeaderEncoder.scala | 47 ---- .../network/netty/server/BlockServer.scala | 162 ----------- .../BlockServerChannelInitializer.scala | 40 --- .../netty/server/BlockServerHandler.scala | 140 ---------- .../network/nio/NioBlockTransferService.scala | 51 ++-- .../apache/spark/serializer/Serializer.scala | 6 +- .../shuffle/FileShuffleBlockManager.scala | 4 +- .../shuffle/IndexShuffleBlockManager.scala | 2 +- .../spark/shuffle/ShuffleBlockManager.scala | 3 +- .../apache/spark/storage/BlockManager.scala | 52 ++-- .../storage/BlockNotFoundException.scala | 1 - .../storage/ShuffleBlockFetcherIterator.scala | 135 ++++++--- .../scala/org/apache/spark/util/Utils.scala | 1 - .../org/apache/spark/ShuffleNettySuite.scala | 4 +- .../netty/ServerClientIntegrationSuite.scala | 161 ----------- .../BlockFetchingClientHandlerSuite.scala | 106 ------- .../server/BlockHeaderEncoderSuite.scala | 64 ----- .../server/BlockServerHandlerSuite.scala | 107 ------- .../spark/serializer/TestSerializer.scala | 60 ++++ .../hash/HashShuffleManagerSuite.scala | 8 +- .../ShuffleBlockFetcherIteratorSuite.scala | 261 +++++++++++------- network/common/pom.xml | 94 +++++++ .../spark/network/TransportContext.java | 117 ++++++++ .../buffer/FileSegmentManagedBuffer.java | 154 +++++++++++ .../spark/network/buffer/ManagedBuffer.java | 71 +++++ .../network/buffer/NettyManagedBuffer.java | 76 +++++ .../network/buffer/NioManagedBuffer.java | 75 +++++ .../client/ChunkFetchFailureException.java | 21 +- .../network/client/ChunkReceivedCallback.java | 47 ++++ .../network/client/RpcResponseCallback.java | 22 +- .../spark/network/client/TransportClient.java | 159 +++++++++++ .../client/TransportClientFactory.java | 182 ++++++++++++ .../client/TransportResponseHandler.java | 167 +++++++++++ .../network/protocol/ChunkFetchFailure.java | 76 +++++ .../network/protocol/ChunkFetchRequest.java | 66 +++++ .../network/protocol/ChunkFetchSuccess.java | 80 ++++++ .../spark/network/protocol/Encodable.java | 41 +++ .../spark/network/protocol/Message.java | 58 ++++ .../network/protocol/MessageDecoder.java | 70 +++++ .../network/protocol/MessageEncoder.java | 80 ++++++ .../network/protocol/RequestMessage.java | 10 +- .../network/protocol/ResponseMessage.java | 14 +- .../spark/network/protocol/RpcFailure.java | 74 +++++ .../spark/network/protocol/RpcRequest.java | 81 ++++++ .../spark/network/protocol/RpcResponse.java | 72 +++++ .../spark/network/protocol/StreamChunkId.java | 73 +++++ .../network/server/DefaultStreamManager.java | 104 +++++++ .../spark/network/server/MessageHandler.java | 36 +++ .../spark/network/server/RpcHandler.java | 38 +++ .../spark/network/server/StreamManager.java | 52 ++++ .../server/TransportChannelHandler.java | 96 +++++++ .../server/TransportRequestHandler.java | 162 +++++++++++ .../spark/network/server/TransportServer.java | 121 ++++++++ .../spark/network/util/ConfigProvider.java | 52 ++++ .../org/apache/spark/network/util/IOMode.java | 27 ++ .../apache/spark/network/util/JavaUtils.java | 38 +++ .../apache/spark/network/util/NettyUtils.java | 102 +++++++ .../spark/network/util/TransportConf.java | 61 ++++ .../network/ChunkFetchIntegrationSuite.java | 217 +++++++++++++++ .../apache/spark/network/NoOpRpcHandler.java | 28 ++ .../apache/spark/network/ProtocolSuite.java | 86 ++++++ .../spark/network/RpcIntegrationSuite.java | 175 ++++++++++++ .../network/SystemPropertyConfigProvider.java | 34 +++ .../spark/network/TestManagedBuffer.java | 104 +++++++ .../org/apache/spark/network/TestUtils.java | 30 ++ .../network/TransportClientFactorySuite.java | 102 +++++++ .../TransportResponseHandlerSuite.java | 115 ++++++++ pom.xml | 1 + project/MimaExcludes.scala | 5 + 84 files changed, 4431 insertions(+), 1750 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala create mode 100644 network/common/pom.xml create mode 100644 network/common/src/main/java/org/apache/spark/network/TransportContext.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java rename core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala => network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java (67%) create mode 100644 network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java rename core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala => network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java (55%) create mode 100644 network/common/src/main/java/org/apache/spark/network/client/TransportClient.java create mode 100644 network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java create mode 100644 network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/Message.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java rename core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala => network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java (77%) rename core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala => network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java (75%) create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/StreamManager.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/TransportServer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/IOMode.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/TransportConf.java create mode 100644 network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java create mode 100644 network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java create mode 100644 network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java create mode 100644 network/common/src/test/java/org/apache/spark/network/TestUtils.java create mode 100644 network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java diff --git a/core/pom.xml b/core/pom.xml index 5cd21e18e8ca7..8020a2daf81ec 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -44,6 +44,11 @@
+ + org.apache.spark + network + ${project.version} + net.java.dev.jets3t jets3t diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5c076e5f1c11d..6a6dfda363974 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -32,6 +32,7 @@ import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.netty.{NettyBlockTransferService} import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer @@ -272,7 +273,13 @@ object SparkEnv extends Logging { val shuffleMemoryManager = new ShuffleMemoryManager(conf) - val blockTransferService = new NioBlockTransferService(conf, securityManager) + val blockTransferService = + conf.get("spark.shuffle.blockTransferService", "nio").toLowerCase match { + case "netty" => + new NettyBlockTransferService(conf) + case "nio" => + new NioBlockTransferService(conf, securityManager) + } val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index e0e91724271c8..1745d52c81923 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -17,20 +17,20 @@ package org.apache.spark.network -import org.apache.spark.storage.StorageLevel - +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.storage.{BlockId, StorageLevel} +private[spark] trait BlockDataManager { /** - * Interface to get local block data. - * - * @return Some(buffer) if the block exists locally, and None if it doesn't. + * Interface to get local block data. Throws an exception if the block cannot be found or + * cannot be read successfully. */ - def getBlockData(blockId: String): Option[ManagedBuffer] + def getBlockData(blockId: BlockId): ManagedBuffer /** * Put the block locally, using the given storage level. */ - def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit + def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit } diff --git a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala index 34acaa563ca58..645793fde806d 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala @@ -19,19 +19,24 @@ package org.apache.spark.network import java.util.EventListener +import org.apache.spark.network.buffer.ManagedBuffer + /** * Listener callback interface for [[BlockTransferService.fetchBlocks]]. */ +private[spark] trait BlockFetchingListener extends EventListener { /** - * Called once per successfully fetched block. + * Called once per successfully fetched block. After this call returns, data will be released + * automatically. If the data will be passed to another thread, the receiver should retain() + * and release() the buffer on their own, or copy the data to a new buffer. */ def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit /** - * Called upon failures. For each failure, this is called only once (i.e. not once per block). + * Called at least once per block upon failures. */ - def onBlockFetchFailure(exception: Throwable): Unit + def onBlockFetchFailure(blockId: String, exception: Throwable): Unit } 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 84d991fa6808c..b083f465334fe 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -17,13 +17,19 @@ package org.apache.spark.network +import java.io.Closeable +import java.nio.ByteBuffer + import scala.concurrent.{Await, Future} import scala.concurrent.duration.Duration -import org.apache.spark.storage.StorageLevel - +import org.apache.spark.Logging +import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.Utils -abstract class BlockTransferService { +private[spark] +abstract class BlockTransferService extends Closeable with Logging { /** * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch @@ -34,7 +40,7 @@ abstract class BlockTransferService { /** * Tear down the transfer service. */ - def stop(): Unit + def close(): Unit /** * Port number the service is listening on, available only after [[init]] is invoked. @@ -50,9 +56,6 @@ abstract class BlockTransferService { * Fetch a sequence of blocks from a remote node asynchronously, * available only after [[init]] is invoked. * - * Note that [[BlockFetchingListener.onBlockFetchSuccess]] is called once per block, - * while [[BlockFetchingListener.onBlockFetchFailure]] is called once per failure (not per block). - * * Note that this API takes a sequence so the implementation can batch requests, and does not * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as * the data of a block is fetched, rather than waiting for all blocks to be fetched. @@ -69,7 +72,7 @@ abstract class BlockTransferService { def uploadBlock( hostname: String, port: Int, - blockId: String, + blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel): Future[Unit] @@ -83,7 +86,7 @@ abstract class BlockTransferService { val lock = new Object @volatile var result: Either[ManagedBuffer, Throwable] = null fetchBlocks(hostName, port, Seq(blockId), new BlockFetchingListener { - override def onBlockFetchFailure(exception: Throwable): Unit = { + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { lock.synchronized { result = Right(exception) lock.notify() @@ -91,7 +94,10 @@ abstract class BlockTransferService { } override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { lock.synchronized { - result = Left(data) + val ret = ByteBuffer.allocate(data.size.toInt) + ret.put(data.nioByteBuffer()) + ret.flip() + result = Left(new NioManagedBuffer(ret)) lock.notify() } } @@ -123,7 +129,7 @@ abstract class BlockTransferService { def uploadBlockSync( hostname: String, port: Int, - blockId: String, + blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel): Unit = { Await.result(uploadBlock(hostname, port, blockId, blockData, level), Duration.Inf) diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala deleted file mode 100644 index 4211ba4e43b9e..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala +++ /dev/null @@ -1,166 +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.network - -import java.io._ -import java.nio.ByteBuffer -import java.nio.channels.FileChannel -import java.nio.channels.FileChannel.MapMode - -import scala.util.Try - -import com.google.common.io.ByteStreams -import io.netty.buffer.{ByteBufInputStream, ByteBuf} - -import org.apache.spark.util.{ByteBufferInputStream, Utils} - - -/** - * This interface provides an immutable view for data in the form of bytes. The implementation - * should specify how the data is provided: - * - * - FileSegmentManagedBuffer: data backed by part of a file - * - NioByteBufferManagedBuffer: data backed by a NIO ByteBuffer - * - NettyByteBufManagedBuffer: data backed by a Netty ByteBuf - */ -sealed abstract class ManagedBuffer { - // Note that all the methods are defined with parenthesis because their implementations can - // have side effects (io operations). - - /** Number of bytes of the data. */ - def size: Long - - /** - * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the - * returned ByteBuffer should not affect the content of this buffer. - */ - def nioByteBuffer(): ByteBuffer - - /** - * Exposes this buffer's data as an InputStream. The underlying implementation does not - * necessarily check for the length of bytes read, so the caller is responsible for making sure - * it does not go over the limit. - */ - def inputStream(): InputStream -} - - -/** - * A [[ManagedBuffer]] backed by a segment in a file - */ -final class FileSegmentManagedBuffer(val file: File, val offset: Long, val length: Long) - extends ManagedBuffer { - - /** - * Memory mapping is expensive and can destabilize the JVM (SPARK-1145, SPARK-3889). - * Avoid unless there's a good reason not to. - */ - private val MIN_MEMORY_MAP_BYTES = 2 * 1024 * 1024; - - override def size: Long = length - - override def nioByteBuffer(): ByteBuffer = { - var channel: FileChannel = null - try { - channel = new RandomAccessFile(file, "r").getChannel - // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. - if (length < MIN_MEMORY_MAP_BYTES) { - val buf = ByteBuffer.allocate(length.toInt) - channel.position(offset) - while (buf.remaining() != 0) { - if (channel.read(buf) == -1) { - throw new IOException("Reached EOF before filling buffer\n" + - s"offset=$offset\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") - } - } - buf.flip() - buf - } else { - channel.map(MapMode.READ_ONLY, offset, length) - } - } catch { - case e: IOException => - Try(channel.size).toOption match { - case Some(fileLen) => - throw new IOException(s"Error in reading $this (actual file length $fileLen)", e) - case None => - throw new IOException(s"Error in opening $this", e) - } - } finally { - if (channel != null) { - Utils.tryLog(channel.close()) - } - } - } - - override def inputStream(): InputStream = { - var is: FileInputStream = null - try { - is = new FileInputStream(file) - ByteStreams.skipFully(is, offset) - ByteStreams.limit(is, length) - } catch { - case e: IOException => - if (is != null) { - Utils.tryLog(is.close()) - } - Try(file.length).toOption match { - case Some(fileLen) => - throw new IOException(s"Error in reading $this (actual file length $fileLen)", e) - case None => - throw new IOException(s"Error in opening $this", e) - } - case e: Throwable => - if (is != null) { - Utils.tryLog(is.close()) - } - throw e - } - } - - override def toString: String = s"${getClass.getName}($file, $offset, $length)" -} - - -/** - * A [[ManagedBuffer]] backed by [[java.nio.ByteBuffer]]. - */ -final class NioByteBufferManagedBuffer(buf: ByteBuffer) extends ManagedBuffer { - - override def size: Long = buf.remaining() - - override def nioByteBuffer() = buf.duplicate() - - override def inputStream() = new ByteBufferInputStream(buf) -} - - -/** - * A [[ManagedBuffer]] backed by a Netty [[ByteBuf]]. - */ -final class NettyByteBufManagedBuffer(buf: ByteBuf) extends ManagedBuffer { - - override def size: Long = buf.readableBytes() - - override def nioByteBuffer() = buf.nioBuffer() - - override def inputStream() = new ByteBufInputStream(buf) - - // TODO(rxin): Promote this to top level ManagedBuffer interface and add documentation for it. - def release(): Unit = buf.release() -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala new file mode 100644 index 0000000000000..8c5ffd8da6bbb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala @@ -0,0 +1,95 @@ +/* + * 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.network.netty + +import java.nio.ByteBuffer +import java.util + +import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.network.BlockFetchingListener +import org.apache.spark.network.netty.NettyMessages._ +import org.apache.spark.serializer.{JavaSerializer, Serializer} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, TransportClient} +import org.apache.spark.storage.BlockId +import org.apache.spark.util.Utils + +/** + * Responsible for holding the state for a request for a single set of blocks. This assumes that + * the chunks will be returned in the same order as requested, and that there will be exactly + * one chunk per block. + * + * Upon receipt of any block, the listener will be called back. Upon failure part way through, + * the listener will receive a failure callback for each outstanding block. + */ +class NettyBlockFetcher( + serializer: Serializer, + client: TransportClient, + blockIds: Seq[String], + listener: BlockFetchingListener) + extends Logging { + + require(blockIds.nonEmpty) + + private val ser = serializer.newInstance() + + private var streamHandle: ShuffleStreamHandle = _ + + private val chunkCallback = new ChunkReceivedCallback { + // On receipt of a chunk, pass it upwards as a block. + def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions { + listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer) + } + + // On receipt of a failure, fail every block from chunkIndex onwards. + def onFailure(chunkIndex: Int, e: Throwable): Unit = { + blockIds.drop(chunkIndex).foreach { blockId => + listener.onBlockFetchFailure(blockId, e); + } + } + } + + /** Begins the fetching process, calling the listener with every block fetched. */ + def start(): Unit = { + // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle. + client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(), + new RpcResponseCallback { + override def onSuccess(response: Array[Byte]): Unit = { + try { + streamHandle = ser.deserialize[ShuffleStreamHandle](ByteBuffer.wrap(response)) + logTrace(s"Successfully opened block set: $streamHandle! Preparing to fetch chunks.") + + // Immediately request all chunks -- we expect that the total size of the request is + // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]]. + for (i <- 0 until streamHandle.numChunks) { + client.fetchChunk(streamHandle.streamId, i, chunkCallback) + } + } catch { + case e: Exception => + logError("Failed while starting block fetches", e) + blockIds.foreach(blockId => Utils.tryLog(listener.onBlockFetchFailure(blockId, e))) + } + } + + override def onFailure(e: Throwable): Unit = { + logError("Failed while starting block fetches", e) + blockIds.foreach(blockId => Utils.tryLog(listener.onBlockFetchFailure(blockId, e))) + } + }) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala new file mode 100644 index 0000000000000..02c657e1d61b5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -0,0 +1,76 @@ +/* + * 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.network.netty + +import java.nio.ByteBuffer + +import org.apache.spark.Logging +import org.apache.spark.network.BlockDataManager +import org.apache.spark.serializer.Serializer +import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} +import org.apache.spark.network.client.{TransportClient, RpcResponseCallback} +import org.apache.spark.network.server.{DefaultStreamManager, RpcHandler} +import org.apache.spark.storage.{StorageLevel, BlockId} + +import scala.collection.JavaConversions._ + +object NettyMessages { + + /** Request to read a set of blocks. Returns [[ShuffleStreamHandle]] to identify the stream. */ + case class OpenBlocks(blockIds: Seq[BlockId]) + + /** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */ + case class UploadBlock(blockId: BlockId, blockData: Array[Byte], level: StorageLevel) + + /** Identifier for a fixed number of chunks to read from a stream created by [[OpenBlocks]]. */ + case class ShuffleStreamHandle(streamId: Long, numChunks: Int) +} + +/** + * Serves requests to open blocks by simply registering one chunk per block requested. + */ +class NettyBlockRpcServer( + serializer: Serializer, + streamManager: DefaultStreamManager, + blockManager: BlockDataManager) + extends RpcHandler with Logging { + + import NettyMessages._ + + override def receive( + client: TransportClient, + messageBytes: Array[Byte], + responseContext: RpcResponseCallback): Unit = { + val ser = serializer.newInstance() + val message = ser.deserialize[AnyRef](ByteBuffer.wrap(messageBytes)) + logTrace(s"Received request: $message") + + message match { + case OpenBlocks(blockIds) => + val blocks: Seq[ManagedBuffer] = blockIds.map(blockManager.getBlockData) + val streamId = streamManager.registerStream(blocks.iterator) + logTrace(s"Registered streamId $streamId with ${blocks.size} buffers") + responseContext.onSuccess( + ser.serialize(new ShuffleStreamHandle(streamId, blocks.size)).array()) + + case UploadBlock(blockId, blockData, level) => + blockManager.putBlockData(blockId, new NioManagedBuffer(ByteBuffer.wrap(blockData)), level) + responseContext.onSuccess(new Array[Byte](0)) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala new file mode 100644 index 0000000000000..38a3e945155e8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -0,0 +1,111 @@ +/* + * 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.network.netty + +import scala.concurrent.{Promise, Future} + +import org.apache.spark.SparkConf +import org.apache.spark.network._ +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.client.{RpcResponseCallback, TransportClient, TransportClientFactory} +import org.apache.spark.network.netty.NettyMessages.UploadBlock +import org.apache.spark.network.server._ +import org.apache.spark.network.util.{ConfigProvider, TransportConf} +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.Utils + +/** + * A BlockTransferService that uses Netty to fetch a set of blocks at at time. + */ +class NettyBlockTransferService(conf: SparkConf) extends BlockTransferService { + // TODO: Don't use Java serialization, use a more cross-version compatible serialization format. + val serializer = new JavaSerializer(conf) + + // Create a TransportConfig using SparkConf. + private[this] val transportConf = new TransportConf( + new ConfigProvider { override def get(name: String) = conf.get(name) }) + + private[this] var transportContext: TransportContext = _ + private[this] var server: TransportServer = _ + private[this] var clientFactory: TransportClientFactory = _ + + override def init(blockDataManager: BlockDataManager): Unit = { + val streamManager = new DefaultStreamManager + val rpcHandler = new NettyBlockRpcServer(serializer, streamManager, blockDataManager) + transportContext = new TransportContext(transportConf, streamManager, rpcHandler) + clientFactory = transportContext.createClientFactory() + server = transportContext.createServer() + } + + override def fetchBlocks( + hostname: String, + port: Int, + blockIds: Seq[String], + listener: BlockFetchingListener): Unit = { + try { + val client = clientFactory.createClient(hostname, port) + new NettyBlockFetcher(serializer, client, blockIds, listener).start() + } catch { + case e: Exception => + logError("Exception while beginning fetchBlocks", e) + blockIds.foreach(listener.onBlockFetchFailure(_, e)) + } + } + + override def hostName: String = Utils.localHostName() + + override def port: Int = server.getPort + + override def uploadBlock( + hostname: String, + port: Int, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel): Future[Unit] = { + val result = Promise[Unit]() + val client = clientFactory.createClient(hostname, port) + + // Convert or copy nio buffer into array in order to serialize it. + val nioBuffer = blockData.nioByteBuffer() + val array = if (nioBuffer.hasArray) { + nioBuffer.array() + } else { + val data = new Array[Byte](nioBuffer.remaining()) + nioBuffer.get(data) + data + } + + val ser = serializer.newInstance() + client.sendRpc(ser.serialize(new UploadBlock(blockId, array, level)).array(), + new RpcResponseCallback { + override def onSuccess(response: Array[Byte]): Unit = { + logTrace(s"Successfully uploaded block $blockId") + result.success() + } + override def onFailure(e: Throwable): Unit = { + logError(s"Error while uploading block $blockId", e) + result.failure(e) + } + }) + + result.future + } + + override def close(): Unit = server.close() +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala deleted file mode 100644 index b5870152c5a64..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala +++ /dev/null @@ -1,59 +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.network.netty - -import org.apache.spark.SparkConf - -/** - * A central location that tracks all the settings we exposed to users. - */ -private[spark] -class NettyConfig(conf: SparkConf) { - - /** Port the server listens on. Default to a random port. */ - private[netty] val serverPort = conf.getInt("spark.shuffle.io.port", 0) - - /** IO mode: nio, oio, epoll, or auto (try epoll first and then nio). */ - private[netty] val ioMode = conf.get("spark.shuffle.io.mode", "nio").toLowerCase - - /** Connect timeout in secs. Default 60 secs. */ - private[netty] val connectTimeoutMs = conf.getInt("spark.shuffle.io.connectionTimeout", 60) * 1000 - - /** - * Percentage of the desired amount of time spent for I/O in the child event loops. - * Only applicable in nio and epoll. - */ - private[netty] val ioRatio = conf.getInt("spark.shuffle.io.netty.ioRatio", 80) - - /** Requested maximum length of the queue of incoming connections. */ - private[netty] val backLog: Option[Int] = conf.getOption("spark.shuffle.io.backLog").map(_.toInt) - - /** - * Receive buffer size (SO_RCVBUF). - * Note: the optimal size for receive buffer and send buffer should be - * latency * network_bandwidth. - * Assuming latency = 1ms, network_bandwidth = 10Gbps - * buffer size should be ~ 1.25MB - */ - private[netty] val receiveBuf: Option[Int] = - conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) - - /** Send buffer size (SO_SNDBUF). */ - private[netty] val sendBuf: Option[Int] = - conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala deleted file mode 100644 index 3ab13b96d7f00..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala +++ /dev/null @@ -1,132 +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.network.netty.client - -import java.util.concurrent.TimeoutException - -import com.google.common.base.Charsets.UTF_8 -import io.netty.bootstrap.Bootstrap -import io.netty.buffer.PooledByteBufAllocator -import io.netty.channel.socket.SocketChannel -import io.netty.channel.{ChannelFutureListener, ChannelFuture, ChannelInitializer, ChannelOption} -import io.netty.handler.codec.LengthFieldBasedFrameDecoder -import io.netty.handler.codec.string.StringEncoder - -import org.apache.spark.Logging - -/** - * Client for fetching data blocks from [[org.apache.spark.network.netty.server.BlockServer]]. - * Use [[BlockFetchingClientFactory]] to instantiate this client. - * - * The constructor blocks until a connection is successfully established. - * - * See [[org.apache.spark.network.netty.server.BlockServer]] for client/server protocol. - * - * Concurrency: thread safe and can be called from multiple threads. - */ -@throws[TimeoutException] -private[spark] -class BlockFetchingClient(factory: BlockFetchingClientFactory, hostname: String, port: Int) - extends Logging { - - private val handler = new BlockFetchingClientHandler - - /** Netty Bootstrap for creating the TCP connection. */ - private val bootstrap: Bootstrap = { - val b = new Bootstrap - b.group(factory.workerGroup) - .channel(factory.socketChannelClass) - // Use pooled buffers to reduce temporary buffer allocation - .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - // Disable Nagle's Algorithm since we don't want packets to wait - .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) - .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) - .option[Integer](ChannelOption.CONNECT_TIMEOUT_MILLIS, factory.conf.connectTimeoutMs) - - b.handler(new ChannelInitializer[SocketChannel] { - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("encoder", new StringEncoder(UTF_8)) - // maxFrameLength = 2G, lengthFieldOffset = 0, lengthFieldLength = 4 - .addLast("framedLengthDecoder", new LengthFieldBasedFrameDecoder(Int.MaxValue, 0, 4)) - .addLast("handler", handler) - } - }) - b - } - - /** Netty ChannelFuture for the connection. */ - private val cf: ChannelFuture = bootstrap.connect(hostname, port) - if (!cf.awaitUninterruptibly(factory.conf.connectTimeoutMs)) { - throw new TimeoutException( - s"Connecting to $hostname:$port timed out (${factory.conf.connectTimeoutMs} ms)") - } - - /** - * Ask the remote server for a sequence of blocks, and execute the callback. - * - * Note that this is asynchronous and returns immediately. Upstream caller should throttle the - * rate of fetching; otherwise we could run out of memory. - * - * @param blockIds sequence of block ids to fetch. - * @param listener callback to fire on fetch success / failure. - */ - def fetchBlocks(blockIds: Seq[String], listener: BlockClientListener): Unit = { - // It's best to limit the number of "write" calls since it needs to traverse the whole pipeline. - // It's also best to limit the number of "flush" calls since it requires system calls. - // Let's concatenate the string and then call writeAndFlush once. - // This is also why this implementation might be more efficient than multiple, separate - // fetch block calls. - var startTime: Long = 0 - logTrace { - startTime = System.nanoTime - s"Sending request $blockIds to $hostname:$port" - } - - blockIds.foreach { blockId => - handler.addRequest(blockId, listener) - } - - val writeFuture = cf.channel().writeAndFlush(blockIds.mkString("\n") + "\n") - writeFuture.addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture): Unit = { - if (future.isSuccess) { - logTrace { - val timeTaken = (System.nanoTime - startTime).toDouble / 1000000 - s"Sending request $blockIds to $hostname:$port took $timeTaken ms" - } - } else { - // Fail all blocks. - val errorMsg = - s"Failed to send request $blockIds to $hostname:$port: ${future.cause.getMessage}" - logError(errorMsg, future.cause) - blockIds.foreach { blockId => - listener.onFetchFailure(blockId, errorMsg) - handler.removeRequest(blockId) - } - } - } - }) - } - - def waitForClose(): Unit = { - cf.channel().closeFuture().sync() - } - - def close(): Unit = cf.channel().close() -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala deleted file mode 100644 index 2b28402c52b49..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala +++ /dev/null @@ -1,99 +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.network.netty.client - -import io.netty.channel.epoll.{EpollEventLoopGroup, EpollSocketChannel} -import io.netty.channel.nio.NioEventLoopGroup -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.nio.NioSocketChannel -import io.netty.channel.socket.oio.OioSocketChannel -import io.netty.channel.{EventLoopGroup, Channel} - -import org.apache.spark.SparkConf -import org.apache.spark.network.netty.NettyConfig -import org.apache.spark.util.Utils - -/** - * Factory for creating [[BlockFetchingClient]] by using createClient. This factory reuses - * the worker thread pool for Netty. - * - * Concurrency: createClient is safe to be called from multiple threads concurrently. - */ -private[spark] -class BlockFetchingClientFactory(val conf: NettyConfig) { - - def this(sparkConf: SparkConf) = this(new NettyConfig(sparkConf)) - - /** A thread factory so the threads are named (for debugging). */ - val threadFactory = Utils.namedThreadFactory("spark-shuffle-client") - - /** The following two are instantiated by the [[init]] method, depending ioMode. */ - var socketChannelClass: Class[_ <: Channel] = _ - var workerGroup: EventLoopGroup = _ - - init() - - /** Initialize [[socketChannelClass]] and [[workerGroup]] based on ioMode. */ - private def init(): Unit = { - def initOio(): Unit = { - socketChannelClass = classOf[OioSocketChannel] - workerGroup = new OioEventLoopGroup(0, threadFactory) - } - def initNio(): Unit = { - socketChannelClass = classOf[NioSocketChannel] - workerGroup = new NioEventLoopGroup(0, threadFactory) - } - def initEpoll(): Unit = { - socketChannelClass = classOf[EpollSocketChannel] - workerGroup = new EpollEventLoopGroup(0, threadFactory) - } - - conf.ioMode match { - case "nio" => initNio() - case "oio" => initOio() - case "epoll" => initEpoll() - case "auto" => - // For auto mode, first try epoll (only available on Linux), then nio. - try { - initEpoll() - } catch { - // TODO: Should we log the throwable? But that always happen on non-Linux systems. - // Perhaps the right thing to do is to check whether the system is Linux, and then only - // call initEpoll on Linux. - case e: Throwable => initNio() - } - } - } - - /** - * Create a new BlockFetchingClient connecting to the given remote host / port. - * - * This blocks until a connection is successfully established. - * - * Concurrency: This method is safe to call from multiple threads. - */ - def createClient(remoteHost: String, remotePort: Int): BlockFetchingClient = { - new BlockFetchingClient(this, remoteHost, remotePort) - } - - def stop(): Unit = { - if (workerGroup != null) { - workerGroup.shutdownGracefully() - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala deleted file mode 100644 index d9d3f7bef0b20..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala +++ /dev/null @@ -1,104 +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.network.netty.client - -import com.google.common.base.Charsets.UTF_8 -import io.netty.buffer.ByteBuf -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.Logging - - -/** - * Handler that processes server responses. It uses the protocol documented in - * [[org.apache.spark.network.netty.server.BlockServer]]. - * - * Concurrency: thread safe and can be called from multiple threads. - */ -private[client] -class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] with Logging { - - /** Tracks the list of outstanding requests and their listeners on success/failure. */ - private val outstandingRequests = java.util.Collections.synchronizedMap { - new java.util.HashMap[String, BlockClientListener] - } - - def addRequest(blockId: String, listener: BlockClientListener): Unit = { - outstandingRequests.put(blockId, listener) - } - - def removeRequest(blockId: String): Unit = { - outstandingRequests.remove(blockId) - } - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - val errorMsg = s"Exception in connection from ${ctx.channel.remoteAddress}: ${cause.getMessage}" - logError(errorMsg, cause) - - // Fire the failure callback for all outstanding blocks - outstandingRequests.synchronized { - val iter = outstandingRequests.entrySet().iterator() - while (iter.hasNext) { - val entry = iter.next() - entry.getValue.onFetchFailure(entry.getKey, errorMsg) - } - outstandingRequests.clear() - } - - ctx.close() - } - - override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { - val totalLen = in.readInt() - val blockIdLen = in.readInt() - val blockIdBytes = new Array[Byte](math.abs(blockIdLen)) - in.readBytes(blockIdBytes) - val blockId = new String(blockIdBytes, UTF_8) - val blockSize = totalLen - math.abs(blockIdLen) - 4 - - def server = ctx.channel.remoteAddress.toString - - // blockIdLen is negative when it is an error message. - if (blockIdLen < 0) { - val errorMessageBytes = new Array[Byte](blockSize) - in.readBytes(errorMessageBytes) - val errorMsg = new String(errorMessageBytes, UTF_8) - logTrace(s"Received block $blockId ($blockSize B) with error $errorMsg from $server") - - val listener = outstandingRequests.get(blockId) - if (listener == null) { - // Ignore callback - logWarning(s"Got a response for block $blockId but it is not in our outstanding requests") - } else { - outstandingRequests.remove(blockId) - listener.onFetchFailure(blockId, errorMsg) - } - } else { - logTrace(s"Received block $blockId ($blockSize B) from $server") - - val listener = outstandingRequests.get(blockId) - if (listener == null) { - // Ignore callback - logWarning(s"Got a response for block $blockId but it is not in our outstanding requests") - } else { - outstandingRequests.remove(blockId) - listener.onFetchSuccess(blockId, new ReferenceCountedBuffer(in)) - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala deleted file mode 100644 index 9740ee64d1f2d..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala +++ /dev/null @@ -1,44 +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.network.netty.client - -/** - * A simple iterator that lazily initializes the underlying iterator. - * - * The use case is that sometimes we might have many iterators open at the same time, and each of - * the iterator might initialize its own buffer (e.g. decompression buffer, deserialization buffer). - * This could lead to too many buffers open. If this iterator is used, we lazily initialize those - * buffers. - */ -private[spark] -class LazyInitIterator(createIterator: => Iterator[Any]) extends Iterator[Any] { - - lazy val proxy = createIterator - - override def hasNext: Boolean = { - val gotNext = proxy.hasNext - if (!gotNext) { - close() - } - gotNext - } - - override def next(): Any = proxy.next() - - def close(): Unit = Unit -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala deleted file mode 100644 index ea1abf5eccc26..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import java.io.InputStream -import java.nio.ByteBuffer - -import io.netty.buffer.{ByteBuf, ByteBufInputStream} - - -/** - * A buffer abstraction based on Netty's ByteBuf so we don't expose Netty. - * This is a Scala value class. - * - * The buffer's life cycle is NOT managed by the JVM, and thus requiring explicit declaration of - * reference by the retain method and release method. - */ -private[spark] -class ReferenceCountedBuffer(val underlying: ByteBuf) extends AnyVal { - - /** Return the nio ByteBuffer view of the underlying buffer. */ - def byteBuffer(): ByteBuffer = underlying.nioBuffer - - /** Creates a new input stream that starts from the current position of the buffer. */ - def inputStream(): InputStream = new ByteBufInputStream(underlying) - - /** Increment the reference counter by one. */ - def retain(): Unit = underlying.retain() - - /** Decrement the reference counter by one and release the buffer if the ref count is 0. */ - def release(): Unit = underlying.release() -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala deleted file mode 100644 index 8e4dda4ef8595..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import io.netty.buffer.ByteBuf -import io.netty.channel.ChannelHandlerContext -import io.netty.handler.codec.MessageToByteEncoder - -/** - * A simple encoder for BlockHeader. See [[BlockServer]] for the server to client protocol. - */ -private[server] -class BlockHeaderEncoder extends MessageToByteEncoder[BlockHeader] { - override def encode(ctx: ChannelHandlerContext, msg: BlockHeader, out: ByteBuf): Unit = { - // message = message length (4 bytes) + block id length (4 bytes) + block id + block data - // message length = block id length (4 bytes) + size of block id + size of block data - val blockIdBytes = msg.blockId.getBytes - msg.error match { - case Some(errorMsg) => - val errorBytes = errorMsg.getBytes - out.writeInt(4 + blockIdBytes.length + errorBytes.size) - out.writeInt(-blockIdBytes.length) // use negative block id length to represent errors - out.writeBytes(blockIdBytes) // next is blockId itself - out.writeBytes(errorBytes) // error message - case None => - out.writeInt(4 + blockIdBytes.length + msg.blockSize) - out.writeInt(blockIdBytes.length) // First 4 bytes is blockId length - out.writeBytes(blockIdBytes) // next is blockId itself - // msg of size blockSize will be written by ServerHandler - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala deleted file mode 100644 index 9194c7ced3547..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala +++ /dev/null @@ -1,162 +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.network.netty.server - -import java.net.InetSocketAddress - -import com.google.common.base.Charsets.UTF_8 -import io.netty.bootstrap.ServerBootstrap -import io.netty.buffer.PooledByteBufAllocator -import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} -import io.netty.channel.epoll.{EpollEventLoopGroup, EpollServerSocketChannel} -import io.netty.channel.nio.NioEventLoopGroup -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.SocketChannel -import io.netty.channel.socket.nio.NioServerSocketChannel -import io.netty.channel.socket.oio.OioServerSocketChannel -import io.netty.handler.codec.LineBasedFrameDecoder -import io.netty.handler.codec.string.StringDecoder - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.network.netty.NettyConfig -import org.apache.spark.storage.BlockDataProvider -import org.apache.spark.util.Utils - - -/** - * Server for serving Spark data blocks. - * This should be used together with [[org.apache.spark.network.netty.client.BlockFetchingClient]]. - * - * Protocol for requesting blocks (client to server): - * One block id per line, e.g. to request 3 blocks: "block1\nblock2\nblock3\n" - * - * Protocol for sending blocks (server to client): - * frame-length (4 bytes), block-id-length (4 bytes), block-id, block-data. - * - * frame-length should not include the length of itself. - * If block-id-length is negative, then this is an error message rather than block-data. The real - * length is the absolute value of the frame-length. - * - */ -private[spark] -class BlockServer(conf: NettyConfig, dataProvider: BlockDataProvider) extends Logging { - - def this(sparkConf: SparkConf, dataProvider: BlockDataProvider) = { - this(new NettyConfig(sparkConf), dataProvider) - } - - def port: Int = _port - - def hostName: String = _hostName - - private var _port: Int = conf.serverPort - private var _hostName: String = "" - private var bootstrap: ServerBootstrap = _ - private var channelFuture: ChannelFuture = _ - - init() - - /** Initialize the server. */ - private def init(): Unit = { - bootstrap = new ServerBootstrap - val bossThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-boss") - val workerThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-worker") - - // Use only one thread to accept connections, and 2 * num_cores for worker. - def initNio(): Unit = { - val bossGroup = new NioEventLoopGroup(1, bossThreadFactory) - val workerGroup = new NioEventLoopGroup(0, workerThreadFactory) - workerGroup.setIoRatio(conf.ioRatio) - bootstrap.group(bossGroup, workerGroup).channel(classOf[NioServerSocketChannel]) - } - def initOio(): Unit = { - val bossGroup = new OioEventLoopGroup(1, bossThreadFactory) - val workerGroup = new OioEventLoopGroup(0, workerThreadFactory) - bootstrap.group(bossGroup, workerGroup).channel(classOf[OioServerSocketChannel]) - } - def initEpoll(): Unit = { - val bossGroup = new EpollEventLoopGroup(1, bossThreadFactory) - val workerGroup = new EpollEventLoopGroup(0, workerThreadFactory) - workerGroup.setIoRatio(conf.ioRatio) - bootstrap.group(bossGroup, workerGroup).channel(classOf[EpollServerSocketChannel]) - } - - conf.ioMode match { - case "nio" => initNio() - case "oio" => initOio() - case "epoll" => initEpoll() - case "auto" => - // For auto mode, first try epoll (only available on Linux), then nio. - try { - initEpoll() - } catch { - // TODO: Should we log the throwable? But that always happen on non-Linux systems. - // Perhaps the right thing to do is to check whether the system is Linux, and then only - // call initEpoll on Linux. - case e: Throwable => initNio() - } - } - - // Use pooled buffers to reduce temporary buffer allocation - bootstrap.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - bootstrap.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - - // Various (advanced) user-configured settings. - conf.backLog.foreach { backLog => - bootstrap.option[java.lang.Integer](ChannelOption.SO_BACKLOG, backLog) - } - conf.receiveBuf.foreach { receiveBuf => - bootstrap.option[java.lang.Integer](ChannelOption.SO_RCVBUF, receiveBuf) - } - conf.sendBuf.foreach { sendBuf => - bootstrap.option[java.lang.Integer](ChannelOption.SO_SNDBUF, sendBuf) - } - - bootstrap.childHandler(new ChannelInitializer[SocketChannel] { - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(UTF_8)) - .addLast("blockHeaderEncoder", new BlockHeaderEncoder) - .addLast("handler", new BlockServerHandler(dataProvider)) - } - }) - - channelFuture = bootstrap.bind(new InetSocketAddress(_port)) - channelFuture.sync() - - val addr = channelFuture.channel.localAddress.asInstanceOf[InetSocketAddress] - _port = addr.getPort - _hostName = addr.getHostName - } - - /** Shutdown the server. */ - def stop(): Unit = { - if (channelFuture != null) { - channelFuture.channel().close().awaitUninterruptibly() - channelFuture = null - } - if (bootstrap != null && bootstrap.group() != null) { - bootstrap.group().shutdownGracefully() - } - if (bootstrap != null && bootstrap.childGroup() != null) { - bootstrap.childGroup().shutdownGracefully() - } - bootstrap = null - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala deleted file mode 100644 index 188154d51d0eb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala +++ /dev/null @@ -1,40 +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.network.netty.server - -import com.google.common.base.Charsets.UTF_8 -import io.netty.channel.ChannelInitializer -import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.LineBasedFrameDecoder -import io.netty.handler.codec.string.StringDecoder - -import org.apache.spark.storage.BlockDataProvider - -/** Channel initializer that sets up the pipeline for the BlockServer. */ -private[netty] -class BlockServerChannelInitializer(dataProvider: BlockDataProvider) - extends ChannelInitializer[SocketChannel] { - - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(UTF_8)) - .addLast("blockHeaderEncoder", new BlockHeaderEncoder) - .addLast("handler", new BlockServerHandler(dataProvider)) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala deleted file mode 100644 index 40dd5e5d1a2ac..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala +++ /dev/null @@ -1,140 +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.network.netty.server - -import java.io.FileInputStream -import java.nio.ByteBuffer -import java.nio.channels.FileChannel - -import io.netty.buffer.Unpooled -import io.netty.channel._ - -import org.apache.spark.Logging -import org.apache.spark.storage.{FileSegment, BlockDataProvider} - - -/** - * A handler that processes requests from clients and writes block data back. - * - * The messages should have been processed by a LineBasedFrameDecoder and a StringDecoder first - * so channelRead0 is called once per line (i.e. per block id). - */ -private[server] -class BlockServerHandler(dataProvider: BlockDataProvider) - extends SimpleChannelInboundHandler[String] with Logging { - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - logError(s"Exception in connection from ${ctx.channel.remoteAddress}", cause) - ctx.close() - } - - override def channelRead0(ctx: ChannelHandlerContext, blockId: String): Unit = { - def client = ctx.channel.remoteAddress.toString - - // A helper function to send error message back to the client. - def respondWithError(error: String): Unit = { - ctx.writeAndFlush(new BlockHeader(-1, blockId, Some(error))).addListener( - new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (!future.isSuccess) { - // TODO: Maybe log the success case as well. - logError(s"Error sending error back to $client", future.cause) - ctx.close() - } - } - } - ) - } - - def writeFileSegment(segment: FileSegment): Unit = { - // Send error message back if the block is too large. Even though we are capable of sending - // large (2G+) blocks, the receiving end cannot handle it so let's fail fast. - // Once we fixed the receiving end to be able to process large blocks, this should be removed. - // Also make sure we update BlockHeaderEncoder to support length > 2G. - - // See [[BlockHeaderEncoder]] for the way length is encoded. - if (segment.length + blockId.length + 4 > Int.MaxValue) { - respondWithError(s"Block $blockId size ($segment.length) greater than 2G") - return - } - - var fileChannel: FileChannel = null - try { - fileChannel = new FileInputStream(segment.file).getChannel - } catch { - case e: Exception => - logError( - s"Error opening channel for $blockId in ${segment.file} for request from $client", e) - respondWithError(e.getMessage) - } - - // Found the block. Send it back. - if (fileChannel != null) { - // Write the header and block data. In the case of failures, the listener on the block data - // write should close the connection. - ctx.write(new BlockHeader(segment.length.toInt, blockId)) - - val region = new DefaultFileRegion(fileChannel, segment.offset, segment.length) - ctx.writeAndFlush(region).addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (future.isSuccess) { - logTrace(s"Sent block $blockId (${segment.length} B) back to $client") - } else { - logError(s"Error sending block $blockId to $client; closing connection", future.cause) - ctx.close() - } - } - }) - } - } - - def writeByteBuffer(buf: ByteBuffer): Unit = { - ctx.write(new BlockHeader(buf.remaining, blockId)) - ctx.writeAndFlush(Unpooled.wrappedBuffer(buf)).addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (future.isSuccess) { - logTrace(s"Sent block $blockId (${buf.remaining} B) back to $client") - } else { - logError(s"Error sending block $blockId to $client; closing connection", future.cause) - ctx.close() - } - } - }) - } - - logTrace(s"Received request from $client to fetch block $blockId") - - var blockData: Either[FileSegment, ByteBuffer] = null - - // First make sure we can find the block. If not, send error back to the user. - try { - blockData = dataProvider.getBlockData(blockId) - } catch { - case e: Exception => - logError(s"Error opening block $blockId for request from $client", e) - respondWithError(e.getMessage) - return - } - - blockData match { - case Left(segment) => writeFileSegment(segment) - case Right(buf) => writeByteBuffer(buf) - } - - } // end of channelRead0 -} diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index e3113205bebdc..11793ea92adb1 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -19,12 +19,13 @@ package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.concurrent.Future - -import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} import org.apache.spark.network._ +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} + +import scala.concurrent.Future /** @@ -71,7 +72,7 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa /** * Tear down the transfer service. */ - override def stop(): Unit = { + override def close(): Unit = { if (cm != null) { cm.stop() } @@ -95,27 +96,34 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa future.onSuccess { case message => val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + // SPARK-4064: In some cases(eg. Remote block was removed) blockMessageArray may be empty. if (blockMessageArray.isEmpty) { - listener.onBlockFetchFailure( - new SparkException(s"Received empty message from $cmId")) + blockIds.foreach { id => + listener.onBlockFetchFailure(id, new SparkException(s"Received empty message from $cmId")) + } } else { - for (blockMessage <- blockMessageArray) { + for (blockMessage: BlockMessage <- blockMessageArray) { val msgType = blockMessage.getType if (msgType != BlockMessage.TYPE_GOT_BLOCK) { - listener.onBlockFetchFailure( - new SparkException(s"Unexpected message ${msgType} received from $cmId")) + if (blockMessage.getId != null) { + listener.onBlockFetchFailure(blockMessage.getId.toString, + new SparkException(s"Unexpected message $msgType received from $cmId")) + } } else { val blockId = blockMessage.getId + val networkSize = blockMessage.getData.limit() listener.onBlockFetchSuccess( - blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + blockId.toString, new NioManagedBuffer(blockMessage.getData)) } } } }(cm.futureExecContext) future.onFailure { case exception => - listener.onBlockFetchFailure(exception) + blockIds.foreach { blockId => + listener.onBlockFetchFailure(blockId, exception) + } }(cm.futureExecContext) } @@ -127,12 +135,12 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa override def uploadBlock( hostname: String, port: Int, - blockId: String, + blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel) : Future[Unit] = { checkInit() - val msg = PutBlock(BlockId(blockId), blockData.nioByteBuffer(), level) + val msg = PutBlock(blockId, blockData.nioByteBuffer(), level) val blockMessageArray = new BlockMessageArray(BlockMessage.fromPutBlock(msg)) val remoteCmId = new ConnectionManagerId(hostName, port) val reply = cm.sendMessageReliably(remoteCmId, blockMessageArray.toBufferMessage) @@ -154,10 +162,9 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) Some(new BlockMessageArray(responseMessages).toBufferMessage) } catch { - case e: Exception => { + case e: Exception => logError("Exception handling buffer message", e) Some(Message.createErrorMessage(e, msg.id)) - } } case otherMessage: Any => @@ -172,13 +179,13 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa case BlockMessage.TYPE_PUT_BLOCK => val msg = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) logDebug("Received [" + msg + "]") - putBlock(msg.id.toString, msg.data, msg.level) + putBlock(msg.id, msg.data, msg.level) None case BlockMessage.TYPE_GET_BLOCK => val msg = new GetBlock(blockMessage.getId) logDebug("Received [" + msg + "]") - val buffer = getBlock(msg.id.toString) + val buffer = getBlock(msg.id) if (buffer == null) { return None } @@ -188,20 +195,20 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa } } - private def putBlock(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + private def putBlock(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel) { val startTimeMs = System.currentTimeMillis() logDebug("PutBlock " + blockId + " started from " + startTimeMs + " with data: " + bytes) - blockDataManager.putBlockData(blockId, new NioByteBufferManagedBuffer(bytes), level) + blockDataManager.putBlockData(blockId, new NioManagedBuffer(bytes), level) logDebug("PutBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + " with data size: " + bytes.limit) } - private def getBlock(blockId: String): ByteBuffer = { + private def getBlock(blockId: BlockId): ByteBuffer = { val startTimeMs = System.currentTimeMillis() logDebug("GetBlock " + blockId + " started from " + startTimeMs) - val buffer = blockDataManager.getBlockData(blockId).orNull + val buffer = blockDataManager.getBlockData(blockId) logDebug("GetBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + " and got buffer " + buffer) - if (buffer == null) null else buffer.nioByteBuffer() + buffer.nioByteBuffer() } } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index a9144cdd97b8c..ca6e971d227fb 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -17,14 +17,14 @@ package org.apache.spark.serializer -import java.io.{ByteArrayOutputStream, EOFException, InputStream, OutputStream} +import java.io._ import java.nio.ByteBuffer import scala.reflect.ClassTag -import org.apache.spark.SparkEnv +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.{ByteBufferInputStream, NextIterator} +import org.apache.spark.util.{Utils, ByteBufferInputStream, NextIterator} /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 439981d232349..1fb5b2c4546bd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -24,9 +24,9 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ -import org.apache.spark.{SparkEnv, SparkConf, Logging} +import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup import org.apache.spark.storage._ diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index b5cd34cacd520..e9805c9c134b5 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import com.google.common.io.ByteStreams import org.apache.spark.SparkEnv -import org.apache.spark.network.{ManagedBuffer, FileSegmentManagedBuffer} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.storage._ /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala index 63863cc0250a3..b521f0c7fc77e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala @@ -18,8 +18,7 @@ package org.apache.spark.shuffle import java.nio.ByteBuffer - -import org.apache.spark.network.ManagedBuffer +import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.storage.ShuffleBlockId private[spark] 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 4cc97923658bc..58510d7232436 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -17,15 +17,13 @@ package org.apache.spark.storage -import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} +import java.io.{BufferedOutputStream, ByteArrayOutputStream, File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} -import scala.concurrent.ExecutionContext.Implicits.global - -import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.concurrent.{Await, Future} +import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ +import scala.concurrent.{Await, Future} import scala.util.Random import akka.actor.{ActorSystem, Props} @@ -35,11 +33,11 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ - private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues @@ -212,21 +210,20 @@ private[spark] class BlockManager( } /** - * Interface to get local block data. - * - * @return Some(buffer) if the block exists locally, and None if it doesn't. + * Interface to get local block data. Throws an exception if the block cannot be found or + * cannot be read successfully. */ - override def getBlockData(blockId: String): Option[ManagedBuffer] = { - val bid = BlockId(blockId) - if (bid.isShuffle) { - Some(shuffleManager.shuffleBlockManager.getBlockData(bid.asInstanceOf[ShuffleBlockId])) + override def getBlockData(blockId: BlockId): ManagedBuffer = { + if (blockId.isShuffle) { + shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) } else { - val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + val blockBytesOpt = doGetLocal(blockId, asBlockResult = false) + .asInstanceOf[Option[ByteBuffer]] if (blockBytesOpt.isDefined) { val buffer = blockBytesOpt.get - Some(new NioByteBufferManagedBuffer(buffer)) + new NioManagedBuffer(buffer) } else { - None + throw new BlockNotFoundException(blockId.toString) } } } @@ -234,8 +231,8 @@ private[spark] class BlockManager( /** * Put the block locally, using the given storage level. */ - override def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit = { - putBytes(BlockId(blockId), data.nioByteBuffer(), level) + override def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit = { + putBytes(blockId, data.nioByteBuffer(), level) } /** @@ -340,17 +337,6 @@ private[spark] class BlockManager( locations } - /** - * A short-circuited method to get blocks directly from disk. This is used for getting - * shuffle blocks. It is safe to do so without a lock on block info since disk store - * never deletes (recent) items. - */ - def getLocalShuffleFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - val buf = shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) - val is = wrapForCompression(blockId, buf.inputStream()) - Some(serializer.newInstance().deserializeStream(is).asIterator) - } - /** * Get block from local block manager. */ @@ -869,9 +855,9 @@ private[spark] class BlockManager( data.rewind() logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") blockTransferService.uploadBlockSync( - peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) - logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %d ms" - .format((System.currentTimeMillis - onePeerStartTime))) + peer.host, peer.port, blockId, new NioManagedBuffer(data), tLevel) + logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %s ms" + .format(System.currentTimeMillis - onePeerStartTime)) peersReplicatedTo += peer peersForReplication -= peer replicationFailed = false @@ -1126,7 +1112,7 @@ private[spark] class BlockManager( } def stop(): Unit = { - blockTransferService.stop() + blockTransferService.close() diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala index 9ef453605f4f1..81f5f2d31dbd8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala @@ -17,5 +17,4 @@ package org.apache.spark.storage - class BlockNotFoundException(blockId: String) extends Exception(s"Block $blockId not found") diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 71b276b5f18e4..0d6f3bf003a9d 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -19,15 +19,13 @@ package org.apache.spark.storage import java.util.concurrent.LinkedBlockingQueue -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet -import scala.collection.mutable.Queue +import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} -import org.apache.spark.{TaskContext, Logging} -import org.apache.spark.network.{ManagedBuffer, BlockFetchingListener, BlockTransferService} +import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} +import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.serializer.Serializer -import org.apache.spark.util.Utils - +import org.apache.spark.util.{CompletionIterator, Utils} /** * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block @@ -88,17 +86,51 @@ final class ShuffleBlockFetcherIterator( */ private[this] val results = new LinkedBlockingQueue[FetchResult] - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight + /** + * Current [[FetchResult]] being processed. We track this so we can release the current buffer + * in case of a runtime exception when processing the current buffer. + */ + private[this] var currentResult: FetchResult = null + + /** + * Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + * the number of bytes in flight is limited to maxBytesInFlight. + */ private[this] val fetchRequests = new Queue[FetchRequest] - // Current bytes in flight from our requests + /** Current bytes in flight from our requests */ private[this] var bytesInFlight = 0L private[this] val shuffleMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + /** + * Whether the iterator is still active. If isZombie is true, the callback interface will no + * longer place fetched blocks into [[results]]. + */ + @volatile private[this] var isZombie = false + initialize() + /** + * Mark the iterator as zombie, and release all buffers that haven't been deserialized yet. + */ + private[this] def cleanup() { + isZombie = true + // Release the current buffer if necessary + if (currentResult != null && !currentResult.failed) { + currentResult.buf.release() + } + + // Release buffers in the results queue + val iter = results.iterator() + while (iter.hasNext) { + val result = iter.next() + if (!result.failed) { + result.buf.release() + } + } + } + private[this] def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) @@ -110,24 +142,23 @@ final class ShuffleBlockFetcherIterator( blockTransferService.fetchBlocks(req.address.host, req.address.port, blockIds, new BlockFetchingListener { - override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { - results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), - () => serializer.newInstance().deserializeStream( - blockManager.wrapForCompression(BlockId(blockId), data.inputStream())).asIterator - )) - shuffleMetrics.remoteBytesRead += data.size - shuffleMetrics.remoteBlocksFetched += 1 - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = { + // Only add the buffer to results queue if the iterator is not zombie, + // i.e. cleanup() has not been called yet. + if (!isZombie) { + // Increment the ref count because we need to pass this to a different thread. + // This needs to be released after use. + buf.retain() + results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), buf)) + shuffleMetrics.remoteBytesRead += buf.size + shuffleMetrics.remoteBlocksFetched += 1 + } + logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } - override def onBlockFetchFailure(e: Throwable): Unit = { + override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) - // Note that there is a chance that some blocks have been fetched successfully, but we - // still add them to the failed queue. This is fine because when the caller see a - // FetchFailedException, it is going to fail the entire task anyway. - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } + results.put(new FetchResult(BlockId(blockId), -1, null)) } } ) @@ -138,7 +169,7 @@ final class ShuffleBlockFetcherIterator( // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 // nodes, rather than blocking on reading output from one node. val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + logDebug("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. @@ -185,26 +216,34 @@ final class ShuffleBlockFetcherIterator( remoteRequests } + /** + * Fetch the local blocks while we are fetching remote blocks. This is ok because + * [[ManagedBuffer]]'s memory is allocated lazily when we create the input stream, so all we + * track in-memory are the ManagedBuffer references themselves. + */ private[this] def fetchLocalBlocks() { - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - for (id <- localBlocks) { + val iter = localBlocks.iterator + while (iter.hasNext) { + val blockId = iter.next() try { + val buf = blockManager.getBlockData(blockId) shuffleMetrics.localBlocksFetched += 1 - results.put(new FetchResult( - id, 0, () => blockManager.getLocalShuffleFromDisk(id, serializer).get)) - logDebug("Got local block " + id) + buf.retain() + results.put(new FetchResult(blockId, 0, buf)) } catch { case e: Exception => + // If we see an exception, stop immediately. logError(s"Error occurred while fetching local blocks", e) - results.put(new FetchResult(id, -1, null)) + results.put(new FetchResult(blockId, -1, null)) return } } } private[this] def initialize(): Unit = { + // Add a task completion callback (called in both success case and failure case) to cleanup. + context.addTaskCompletionListener(_ => cleanup()) + // Split local and remote blocks. val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order @@ -229,7 +268,8 @@ final class ShuffleBlockFetcherIterator( override def next(): (BlockId, Option[Iterator[Any]]) = { numBlocksProcessed += 1 val startFetchWait = System.currentTimeMillis() - val result = results.take() + currentResult = results.take() + val result = currentResult val stopFetchWait = System.currentTimeMillis() shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) if (!result.failed) { @@ -240,7 +280,21 @@ final class ShuffleBlockFetcherIterator( (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { sendRequest(fetchRequests.dequeue()) } - (result.blockId, if (result.failed) None else Some(result.deserialize())) + + val iteratorOpt: Option[Iterator[Any]] = if (result.failed) { + None + } else { + val is = blockManager.wrapForCompression(result.blockId, result.buf.createInputStream()) + val iter = serializer.newInstance().deserializeStream(is).asIterator + Some(CompletionIterator[Any, Iterator[Any]](iter, { + // Once the iterator is exhausted, release the buffer and set currentResult to null + // so we don't release it again in cleanup. + currentResult = null + result.buf.release() + })) + } + + (result.blockId, iteratorOpt) } } @@ -254,7 +308,7 @@ object ShuffleBlockFetcherIterator { * @param blocks Sequence of tuple, where the first element is the block id, * and the second element is the estimated size, used to calculate bytesInFlight. */ - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { + case class FetchRequest(address: BlockManagerId, blocks: Seq[(BlockId, Long)]) { val size = blocks.map(_._2).sum } @@ -262,10 +316,11 @@ object ShuffleBlockFetcherIterator { * Result of a fetch from a remote block. A failure is represented as size == -1. * @param blockId block id * @param size estimated size of the block, used to calculate bytesInFlight. - * Note that this is NOT the exact bytes. - * @param deserialize closure to return the result in the form of an Iterator. + * Note that this is NOT the exact bytes. -1 if failure is present. + * @param buf [[ManagedBuffer]] for the content. null is error. */ - class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { + case class FetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer) { def failed: Boolean = size == -1 + if (failed) assert(buf == null) else assert(buf != null) } } 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 1e881da5114d3..0daab91143e47 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -43,7 +43,6 @@ import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark._ -import org.apache.spark.util.SparkUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index d7b2d2e1e330f..840d8273cb6a8 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -24,10 +24,10 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. override def beforeAll() { - System.setProperty("spark.shuffle.use.netty", "true") + System.setProperty("spark.shuffle.blockTransferService", "netty") } override def afterAll() { - System.clearProperty("spark.shuffle.use.netty") + System.clearProperty("spark.shuffle.blockTransferService") } } diff --git a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala deleted file mode 100644 index 02d0ffc86f58f..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala +++ /dev/null @@ -1,161 +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.network.netty - -import java.io.{RandomAccessFile, File} -import java.nio.ByteBuffer -import java.util.{Collections, HashSet} -import java.util.concurrent.{TimeUnit, Semaphore} - -import scala.collection.JavaConversions._ - -import io.netty.buffer.{ByteBufUtil, Unpooled} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.SparkConf -import org.apache.spark.network.netty.client.{BlockClientListener, ReferenceCountedBuffer, BlockFetchingClientFactory} -import org.apache.spark.network.netty.server.BlockServer -import org.apache.spark.storage.{FileSegment, BlockDataProvider} - - -/** - * Test suite that makes sure the server and the client implementations share the same protocol. - */ -class ServerClientIntegrationSuite extends FunSuite with BeforeAndAfterAll { - - val bufSize = 100000 - var buf: ByteBuffer = _ - var testFile: File = _ - var server: BlockServer = _ - var clientFactory: BlockFetchingClientFactory = _ - - val bufferBlockId = "buffer_block" - val fileBlockId = "file_block" - - val fileContent = new Array[Byte](1024) - scala.util.Random.nextBytes(fileContent) - - override def beforeAll() = { - buf = ByteBuffer.allocate(bufSize) - for (i <- 1 to bufSize) { - buf.put(i.toByte) - } - buf.flip() - - testFile = File.createTempFile("netty-test-file", "txt") - val fp = new RandomAccessFile(testFile, "rw") - fp.write(fileContent) - fp.close() - - server = new BlockServer(new SparkConf, new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { - if (blockId == bufferBlockId) { - Right(buf) - } else if (blockId == fileBlockId) { - Left(new FileSegment(testFile, 10, testFile.length - 25)) - } else { - throw new Exception("Unknown block id " + blockId) - } - } - }) - - clientFactory = new BlockFetchingClientFactory(new SparkConf) - } - - override def afterAll() = { - server.stop() - clientFactory.stop() - } - - /** A ByteBuf for buffer_block */ - lazy val byteBufferBlockReference = Unpooled.wrappedBuffer(buf) - - /** A ByteBuf for file_block */ - lazy val fileBlockReference = Unpooled.wrappedBuffer(fileContent, 10, fileContent.length - 25) - - def fetchBlocks(blockIds: Seq[String]): (Set[String], Set[ReferenceCountedBuffer], Set[String]) = - { - val client = clientFactory.createClient(server.hostName, server.port) - val sem = new Semaphore(0) - val receivedBlockIds = Collections.synchronizedSet(new HashSet[String]) - val errorBlockIds = Collections.synchronizedSet(new HashSet[String]) - val receivedBuffers = Collections.synchronizedSet(new HashSet[ReferenceCountedBuffer]) - - client.fetchBlocks( - blockIds, - new BlockClientListener { - override def onFetchFailure(blockId: String, errorMsg: String): Unit = { - errorBlockIds.add(blockId) - sem.release() - } - - override def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit = { - receivedBlockIds.add(blockId) - data.retain() - receivedBuffers.add(data) - sem.release() - } - } - ) - if (!sem.tryAcquire(blockIds.size, 30, TimeUnit.SECONDS)) { - fail("Timeout getting response from the server") - } - client.close() - (receivedBlockIds.toSet, receivedBuffers.toSet, errorBlockIds.toSet) - } - - test("fetch a ByteBuffer block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId)) - assert(blockIds === Set(bufferBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch a FileSegment block via zero-copy send") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(fileBlockId)) - assert(blockIds === Set(fileBlockId)) - assert(buffers.map(_.underlying) === Set(fileBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch a non-existent block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq("random-block")) - assert(blockIds.isEmpty) - assert(buffers.isEmpty) - assert(failBlockIds === Set("random-block")) - } - - test("fetch both ByteBuffer block and FileSegment block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, fileBlockId)) - assert(blockIds === Set(bufferBlockId, fileBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference, fileBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch both ByteBuffer block and a non-existent block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, "random-block")) - assert(blockIds === Set(bufferBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) - assert(failBlockIds === Set("random-block")) - buffers.foreach(_.release()) - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala deleted file mode 100644 index f629322ff6838..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala +++ /dev/null @@ -1,106 +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.network.netty.client - -import java.nio.ByteBuffer - -import com.google.common.base.Charsets.UTF_8 -import io.netty.buffer.Unpooled -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.{PrivateMethodTester, FunSuite} - - -class BlockFetchingClientHandlerSuite extends FunSuite with PrivateMethodTester { - - test("handling block data (successful fetch)") { - val blockId = "test_block" - val blockData = "blahblahblahblahblah" - val totalLength = 4 + blockId.length + blockData.length - - var parsedBlockId: String = "" - var parsedBlockData: String = "" - val handler = new BlockFetchingClientHandler - handler.addRequest(blockId, - new BlockClientListener { - override def onFetchFailure(blockId: String, errorMsg: String): Unit = ??? - override def onFetchSuccess(bid: String, refCntBuf: ReferenceCountedBuffer): Unit = { - parsedBlockId = bid - val bytes = new Array[Byte](refCntBuf.byteBuffer().remaining) - refCntBuf.byteBuffer().get(bytes) - parsedBlockData = new String(bytes, UTF_8) - } - } - ) - - val outstandingRequests = PrivateMethod[java.util.Map[_, _]]('outstandingRequests) - assert(handler.invokePrivate(outstandingRequests()).size === 1) - - val channel = new EmbeddedChannel(handler) - val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself - buf.putInt(totalLength) - buf.putInt(blockId.length) - buf.put(blockId.getBytes) - buf.put(blockData.getBytes) - buf.flip() - - channel.writeInbound(Unpooled.wrappedBuffer(buf)) - assert(parsedBlockId === blockId) - assert(parsedBlockData === blockData) - - assert(handler.invokePrivate(outstandingRequests()).size === 0) - - channel.close() - } - - test("handling error message (failed fetch)") { - val blockId = "test_block" - val errorMsg = "error erro5r error err4or error3 error6 error erro1r" - val totalLength = 4 + blockId.length + errorMsg.length - - var parsedBlockId: String = "" - var parsedErrorMsg: String = "" - val handler = new BlockFetchingClientHandler - handler.addRequest(blockId, new BlockClientListener { - override def onFetchFailure(bid: String, msg: String) ={ - parsedBlockId = bid - parsedErrorMsg = msg - } - override def onFetchSuccess(bid: String, refCntBuf: ReferenceCountedBuffer) = ??? - }) - - val outstandingRequests = PrivateMethod[java.util.Map[_, _]]('outstandingRequests) - assert(handler.invokePrivate(outstandingRequests()).size === 1) - - val channel = new EmbeddedChannel(handler) - val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself - buf.putInt(totalLength) - buf.putInt(-blockId.length) - buf.put(blockId.getBytes) - buf.put(errorMsg.getBytes) - buf.flip() - - channel.writeInbound(Unpooled.wrappedBuffer(buf)) - assert(parsedBlockId === blockId) - assert(parsedErrorMsg === errorMsg) - - assert(handler.invokePrivate(outstandingRequests()).size === 0) - - channel.close() - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala deleted file mode 100644 index 3f8d0cf8f3ff3..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import com.google.common.base.Charsets.UTF_8 -import io.netty.buffer.ByteBuf -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.FunSuite - -class BlockHeaderEncoderSuite extends FunSuite { - - test("encode normal block data") { - val blockId = "test_block" - val channel = new EmbeddedChannel(new BlockHeaderEncoder) - channel.writeOutbound(new BlockHeader(17, blockId, None)) - val out = channel.readOutbound().asInstanceOf[ByteBuf] - assert(out.readInt() === 4 + blockId.length + 17) - assert(out.readInt() === blockId.length) - - val blockIdBytes = new Array[Byte](blockId.length) - out.readBytes(blockIdBytes) - assert(new String(blockIdBytes, UTF_8) === blockId) - assert(out.readableBytes() === 0) - - channel.close() - } - - test("encode error message") { - val blockId = "error_block" - val errorMsg = "error encountered" - val channel = new EmbeddedChannel(new BlockHeaderEncoder) - channel.writeOutbound(new BlockHeader(17, blockId, Some(errorMsg))) - val out = channel.readOutbound().asInstanceOf[ByteBuf] - assert(out.readInt() === 4 + blockId.length + errorMsg.length) - assert(out.readInt() === -blockId.length) - - val blockIdBytes = new Array[Byte](blockId.length) - out.readBytes(blockIdBytes) - assert(new String(blockIdBytes, UTF_8) === blockId) - - val errorMsgBytes = new Array[Byte](errorMsg.length) - out.readBytes(errorMsgBytes) - assert(new String(errorMsgBytes, UTF_8) === errorMsg) - assert(out.readableBytes() === 0) - - channel.close() - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala deleted file mode 100644 index 3239c710f1639..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala +++ /dev/null @@ -1,107 +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.network.netty.server - -import java.io.{RandomAccessFile, File} -import java.nio.ByteBuffer - -import io.netty.buffer.{Unpooled, ByteBuf} -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler, DefaultFileRegion} -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.FunSuite - -import org.apache.spark.storage.{BlockDataProvider, FileSegment} - - -class BlockServerHandlerSuite extends FunSuite { - - test("ByteBuffer block") { - val expectedBlockId = "test_bytebuffer_block" - val buf = ByteBuffer.allocate(10000) - for (i <- 1 to 10000) { - buf.put(i.toByte) - } - buf.flip() - - val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = Right(buf) - })) - - channel.writeInbound(expectedBlockId) - assert(channel.outboundMessages().size === 2) - - val out1 = channel.readOutbound().asInstanceOf[BlockHeader] - val out2 = channel.readOutbound().asInstanceOf[ByteBuf] - - assert(out1.blockId === expectedBlockId) - assert(out1.blockSize === buf.remaining) - assert(out1.error === None) - - assert(out2.equals(Unpooled.wrappedBuffer(buf))) - - channel.close() - } - - test("FileSegment block via zero-copy") { - val expectedBlockId = "test_file_block" - - // Create random file data - val fileContent = new Array[Byte](1024) - scala.util.Random.nextBytes(fileContent) - val testFile = File.createTempFile("netty-test-file", "txt") - val fp = new RandomAccessFile(testFile, "rw") - fp.write(fileContent) - fp.close() - - val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { - Left(new FileSegment(testFile, 15, testFile.length - 25)) - } - })) - - channel.writeInbound(expectedBlockId) - assert(channel.outboundMessages().size === 2) - - val out1 = channel.readOutbound().asInstanceOf[BlockHeader] - val out2 = channel.readOutbound().asInstanceOf[DefaultFileRegion] - - assert(out1.blockId === expectedBlockId) - assert(out1.blockSize === testFile.length - 25) - assert(out1.error === None) - - assert(out2.count === testFile.length - 25) - assert(out2.position === 15) - } - - test("pipeline exception propagation") { - val blockServerHandler = new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = ??? - }) - val exceptionHandler = new SimpleChannelInboundHandler[String]() { - override def channelRead0(ctx: ChannelHandlerContext, msg: String): Unit = { - throw new Exception("this is an error") - } - } - - val channel = new EmbeddedChannel(exceptionHandler, blockServerHandler) - assert(channel.isOpen) - channel.writeInbound("a message to trigger the error") - assert(!channel.isOpen) - } -} diff --git a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala new file mode 100644 index 0000000000000..0ade1bab18d7e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala @@ -0,0 +1,60 @@ +/* + * 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.serializer + +import java.io.{EOFException, OutputStream, InputStream} +import java.nio.ByteBuffer + +import scala.reflect.ClassTag + + +/** + * A serializer implementation that always return a single element in a deserialization stream. + */ +class TestSerializer extends Serializer { + override def newInstance() = new TestSerializerInstance +} + + +class TestSerializerInstance extends SerializerInstance { + override def serialize[T: ClassTag](t: T): ByteBuffer = ??? + + override def serializeStream(s: OutputStream): SerializationStream = ??? + + override def deserializeStream(s: InputStream) = new TestDeserializationStream + + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = ??? + + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = ??? +} + + +class TestDeserializationStream extends DeserializationStream { + + private var count = 0 + + override def readObject[T: ClassTag](): T = { + count += 1 + if (count == 2) { + throw new EOFException + } + new Object().asInstanceOf[T] + } + + override def close(): Unit = {} +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index ba47fe5e25b9b..6790388f96603 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.FunSuite import org.apache.spark.{SparkEnv, SparkContext, LocalSparkContext, SparkConf} import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.shuffle.FileShuffleBlockManager import org.apache.spark.storage.{ShuffleBlockId, FileSegment} @@ -36,9 +36,9 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { private def checkSegments(expected: FileSegment, buffer: ManagedBuffer) { assert(buffer.isInstanceOf[FileSegmentManagedBuffer]) val segment = buffer.asInstanceOf[FileSegmentManagedBuffer] - assert(expected.file.getCanonicalPath === segment.file.getCanonicalPath) - assert(expected.offset === segment.offset) - assert(expected.length === segment.length) + assert(expected.file.getCanonicalPath === segment.getFile.getCanonicalPath) + assert(expected.offset === segment.getOffset) + assert(expected.length === segment.getLength) } test("consolidated shuffle can write to shuffle group without messing existing offsets/lengths") { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index a8c049d749015..4e502cf65e6b3 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark.storage +import java.util.concurrent.Semaphore + +import scala.concurrent.future +import scala.concurrent.ExecutionContext.Implicits.global import org.apache.spark.{TaskContextImpl, TaskContext} import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} @@ -27,38 +31,64 @@ import org.mockito.stubbing.Answer import org.scalatest.FunSuite +import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.network._ +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.serializer.TestSerializer + class ShuffleBlockFetcherIteratorSuite extends FunSuite { + // Some of the tests are quite tricky because we are testing the cleanup behavior + // in the presence of faults. - test("handle local read failures in BlockManager") { + /** Creates a mock [[BlockTransferService]] that returns data from the given map. */ + private def createMockTransfer(data: Map[BlockId, ManagedBuffer]): BlockTransferService = { val transfer = mock(classOf[BlockTransferService]) - val blockManager = mock(classOf[BlockManager]) - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) - - val optItr = mock(classOf[Option[Iterator[Any]]]) - val answer = new Answer[Option[Iterator[Any]]] { - override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { - throw new Exception + when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val blocks = invocation.getArguments()(2).asInstanceOf[Seq[String]] + val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + + for (blockId <- blocks) { + if (data.contains(BlockId(blockId))) { + listener.onBlockFetchSuccess(blockId, data(BlockId(blockId))) + } else { + listener.onBlockFetchFailure(blockId, new BlockNotFoundException(blockId)) + } + } } + }) + transfer + } + + private val conf = new SparkConf + + test("successful 3 local reads + 2 remote reads") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure blockManager.getBlockData would return the blocks + val localBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer])) + localBlocks.foreach { case (blockId, buf) => + doReturn(buf).when(blockManager).getBlockData(meq(blockId)) } - // 3rd block is going to fail - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + // Make sure remote blocks would return + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val remoteBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 3, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 4, 0) -> mock(classOf[ManagedBuffer]) + ) + + val transfer = createMockTransfer(remoteBlocks) - val bmId = BlockManagerId("test-client", "test-client", 1) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + (localBmId, localBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq), + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq) ) val iterator = new ShuffleBlockFetcherIterator( @@ -66,118 +96,145 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { transfer, blockManager, blocksByAddress, - null, + new TestSerializer, 48 * 1024 * 1024) - // Without exhausting the iterator, the iterator should be lazy and not call - // getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - // the 2nd element of the tuple returned by iterator.next should be defined when - // fetching successfully - assert(iterator.next()._2.isDefined, - "1st element should be defined but is not actually defined") - verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next()._2.isDefined, - "2nd element should be defined but is not actually defined") - verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - // 3rd fetch should be failed - intercept[Exception] { - iterator.next() + // 3 local blocks fetched in initialization + verify(blockManager, times(3)).getBlockData(any()) + + for (i <- 0 until 5) { + assert(iterator.hasNext, s"iterator should have 5 elements but actually has $i elements") + val (blockId, subIterator) = iterator.next() + assert(subIterator.isDefined, + s"iterator should have 5 elements defined but actually has $i elements") + + // Make sure we release the buffer once the iterator is exhausted. + val mockBuf = localBlocks.getOrElse(blockId, remoteBlocks(blockId)) + verify(mockBuf, times(0)).release() + subIterator.get.foreach(_ => Unit) // exhaust the iterator + verify(mockBuf, times(1)).release() } - verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any()) + + // 3 local blocks, and 2 remote blocks + // (but from the same block manager so one call to fetchBlocks) + verify(blockManager, times(3)).getBlockData(any()) + verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any()) } - test("handle local read successes") { - val transfer = mock(classOf[BlockTransferService]) + test("release current unexhausted buffer in case the task completes early") { val blockManager = mock(classOf[BlockManager]) - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure remote blocks would return + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val blocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer]) + ) - val optItr = mock(classOf[Option[Iterator[Any]]]) + // Semaphore to coordinate event sequence in two different threads. + val sem = new Semaphore(0) - // All blocks should be fetched successfully - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + val transfer = mock(classOf[BlockTransferService]) + when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + future { + // Return the first two blocks, and wait till task completion before returning the 3rd one + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 1, 0).toString, blocks(ShuffleBlockId(0, 1, 0))) + sem.acquire() + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 2, 0).toString, blocks(ShuffleBlockId(0, 2, 0))) + } + } + }) - val bmId = BlockManagerId("test-client", "test-client", 1) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) - ) + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) + val taskContext = new TaskContextImpl(0, 0, 0) val iterator = new ShuffleBlockFetcherIterator( - new TaskContextImpl(0, 0, 0), + taskContext, transfer, blockManager, blocksByAddress, - null, + new TestSerializer, 48 * 1024 * 1024) - // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 1st element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 2nd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 3rd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 4th element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 5th element is not actually defined") - - verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any()) + // Exhaust the first block, and then it should be released. + iterator.next()._2.get.foreach(_ => Unit) + verify(blocks(ShuffleBlockId(0, 0, 0)), times(1)).release() + + // Get the 2nd block but do not exhaust the iterator + val subIter = iterator.next()._2.get + + // Complete the task; then the 2nd block buffer should be exhausted + verify(blocks(ShuffleBlockId(0, 1, 0)), times(0)).release() + taskContext.markTaskCompleted() + verify(blocks(ShuffleBlockId(0, 1, 0)), times(1)).release() + + // The 3rd block should not be retained because the iterator is already in zombie state + sem.release() + verify(blocks(ShuffleBlockId(0, 2, 0)), times(0)).retain() + verify(blocks(ShuffleBlockId(0, 2, 0)), times(0)).release() } - test("handle remote fetch failures in BlockTransferService") { + test("fail all blocks if any of the remote request fails") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure remote blocks would return + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val blocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer]) + ) + + // Semaphore to coordinate event sequence in two different threads. + val sem = new Semaphore(0) + val transfer = mock(classOf[BlockTransferService]) when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] - listener.onBlockFetchFailure(new Exception("blah")) + future { + // Return the first block, and then fail. + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) + listener.onBlockFetchFailure( + ShuffleBlockId(0, 1, 0).toString, new BlockNotFoundException("blah")) + listener.onBlockFetchFailure( + ShuffleBlockId(0, 2, 0).toString, new BlockNotFoundException("blah")) + sem.release() + } } }) - val blockManager = mock(classOf[BlockManager]) - - when(blockManager.blockManagerId).thenReturn(BlockManagerId("test-client", "test-client", 1)) - - val blId1 = ShuffleBlockId(0, 0, 0) - val blId2 = ShuffleBlockId(0, 1, 0) - val bmId = BlockManagerId("test-server", "test-server", 1) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, Seq((blId1, 1L), (blId2, 1L)))) + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) + val taskContext = new TaskContextImpl(0, 0, 0) val iterator = new ShuffleBlockFetcherIterator( - new TaskContextImpl(0, 0, 0), + taskContext, transfer, blockManager, blocksByAddress, - null, + new TestSerializer, 48 * 1024 * 1024) - iterator.foreach { case (_, iterOption) => - assert(!iterOption.isDefined) - } + // Continue only after the mock calls onBlockFetchFailure + sem.acquire() + + // The first block should be defined, and the last two are not defined (due to failure) + assert(iterator.next()._2.isDefined === true) + assert(iterator.next()._2.isDefined === false) + assert(iterator.next()._2.isDefined === false) } } diff --git a/network/common/pom.xml b/network/common/pom.xml new file mode 100644 index 0000000000000..e3b7e328701b4 --- /dev/null +++ b/network/common/pom.xml @@ -0,0 +1,94 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.2.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + network + jar + Shuffle Streaming Service + http://spark.apache.org/ + + network + + + + + + io.netty + netty-all + + + org.slf4j + slf4j-api + + + + + com.google.guava + guava + provided + + + + + junit + junit + test + + + log4j + log4j + test + + + org.mockito + mockito-all + test + + + + + + target/java/classes + target/java/test-classes + + + org.apache.maven.plugins + maven-surefire-plugin + 2.17 + + false + + **/Test*.java + **/*Test.java + **/*Suite.java + + + + + + diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java new file mode 100644 index 0000000000000..854aa6685f85f --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java @@ -0,0 +1,117 @@ +/* + * 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.network; + +import io.netty.channel.Channel; +import io.netty.channel.socket.SocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.MessageDecoder; +import org.apache.spark.network.protocol.MessageEncoder; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportChannelHandler; +import org.apache.spark.network.server.TransportRequestHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Contains the context to create a {@link TransportServer}, {@link TransportClientFactory}, and to + * setup Netty Channel pipelines with a {@link org.apache.spark.network.server.TransportChannelHandler}. + * + * There are two communication protocols that the TransportClient provides, control-plane RPCs and + * data-plane "chunk fetching". The handling of the RPCs is performed outside of the scope of the + * TransportContext (i.e., by a user-provided handler), and it is responsible for setting up streams + * which can be streamed through the data plane in chunks using zero-copy IO. + * + * The TransportServer and TransportClientFactory both create a TransportChannelHandler for each + * channel. As each TransportChannelHandler contains a TransportClient, this enables server + * processes to send messages back to the client on an existing channel. + */ +public class TransportContext { + private final Logger logger = LoggerFactory.getLogger(TransportContext.class); + + private final TransportConf conf; + private final StreamManager streamManager; + private final RpcHandler rpcHandler; + + private final MessageEncoder encoder; + private final MessageDecoder decoder; + + public TransportContext(TransportConf conf, StreamManager streamManager, RpcHandler rpcHandler) { + this.conf = conf; + this.streamManager = streamManager; + this.rpcHandler = rpcHandler; + this.encoder = new MessageEncoder(); + this.decoder = new MessageDecoder(); + } + + public TransportClientFactory createClientFactory() { + return new TransportClientFactory(this); + } + + public TransportServer createServer() { + return new TransportServer(this); + } + + /** + * Initializes a client or server Netty Channel Pipeline which encodes/decodes messages and + * has a {@link org.apache.spark.network.server.TransportChannelHandler} to handle request or + * response messages. + * + * @return Returns the created TransportChannelHandler, which includes a TransportClient that can + * be used to communicate on this channel. The TransportClient is directly associated with a + * ChannelHandler to ensure all users of the same channel get the same TransportClient object. + */ + public TransportChannelHandler initializePipeline(SocketChannel channel) { + try { + TransportChannelHandler channelHandler = createChannelHandler(channel); + channel.pipeline() + .addLast("encoder", encoder) + .addLast("frameDecoder", NettyUtils.createFrameDecoder()) + .addLast("decoder", decoder) + // NOTE: Chunks are currently guaranteed to be returned in the order of request, but this + // would require more logic to guarantee if this were not part of the same event loop. + .addLast("handler", channelHandler); + return channelHandler; + } catch (RuntimeException e) { + logger.error("Error while initializing Netty pipeline", e); + throw e; + } + } + + /** + * Creates the server- and client-side handler which is used to handle both RequestMessages and + * ResponseMessages. The channel is expected to have been successfully created, though certain + * properties (such as the remoteAddress()) may not be available yet. + */ + private TransportChannelHandler createChannelHandler(Channel channel) { + TransportResponseHandler responseHandler = new TransportResponseHandler(channel); + TransportClient client = new TransportClient(channel, responseHandler); + TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, + streamManager, rpcHandler); + return new TransportChannelHandler(client, responseHandler, requestHandler); + } + + public TransportConf getConf() { return conf; } +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java new file mode 100644 index 0000000000000..89ed79bc63903 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -0,0 +1,154 @@ +/* + * 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.network.buffer; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; + +import com.google.common.base.Objects; +import com.google.common.io.ByteStreams; +import io.netty.channel.DefaultFileRegion; + +import org.apache.spark.network.util.JavaUtils; + +/** + * A {@link ManagedBuffer} backed by a segment in a file. + */ +public final class FileSegmentManagedBuffer extends ManagedBuffer { + + /** + * Memory mapping is expensive and can destabilize the JVM (SPARK-1145, SPARK-3889). + * Avoid unless there's a good reason not to. + */ + // TODO: Make this configurable + private static final long MIN_MEMORY_MAP_BYTES = 2 * 1024 * 1024; + + private final File file; + private final long offset; + private final long length; + + public FileSegmentManagedBuffer(File file, long offset, long length) { + this.file = file; + this.offset = offset; + this.length = length; + } + + @Override + public long size() { + return length; + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + FileChannel channel = null; + try { + channel = new RandomAccessFile(file, "r").getChannel(); + // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. + if (length < MIN_MEMORY_MAP_BYTES) { + ByteBuffer buf = ByteBuffer.allocate((int) length); + channel.position(offset); + while (buf.remaining() != 0) { + if (channel.read(buf) == -1) { + throw new IOException(String.format("Reached EOF before filling buffer\n" + + "offset=%s\nfile=%s\nbuf.remaining=%s", + offset, file.getAbsoluteFile(), buf.remaining())); + } + } + buf.flip(); + return buf; + } else { + return channel.map(FileChannel.MapMode.READ_ONLY, offset, length); + } + } catch (IOException e) { + try { + if (channel != null) { + long size = channel.size(); + throw new IOException("Error in reading " + this + " (actual file length " + size + ")", + e); + } + } catch (IOException ignored) { + // ignore + } + throw new IOException("Error in opening " + this, e); + } finally { + JavaUtils.closeQuietly(channel); + } + } + + @Override + public InputStream createInputStream() throws IOException { + FileInputStream is = null; + try { + is = new FileInputStream(file); + ByteStreams.skipFully(is, offset); + return ByteStreams.limit(is, length); + } catch (IOException e) { + try { + if (is != null) { + long size = file.length(); + throw new IOException("Error in reading " + this + " (actual file length " + size + ")", + e); + } + } catch (IOException ignored) { + // ignore + } finally { + JavaUtils.closeQuietly(is); + } + throw new IOException("Error in opening " + this, e); + } catch (RuntimeException e) { + JavaUtils.closeQuietly(is); + throw e; + } + } + + @Override + public ManagedBuffer retain() { + return this; + } + + @Override + public ManagedBuffer release() { + return this; + } + + @Override + public Object convertToNetty() throws IOException { + FileChannel fileChannel = new FileInputStream(file).getChannel(); + return new DefaultFileRegion(fileChannel, offset, length); + } + + public File getFile() { return file; } + + public long getOffset() { return offset; } + + public long getLength() { return length; } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("file", file) + .add("offset", offset) + .add("length", length) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java new file mode 100644 index 0000000000000..a415db593a788 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -0,0 +1,71 @@ +/* + * 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.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +/** + * This interface provides an immutable view for data in the form of bytes. The implementation + * should specify how the data is provided: + * + * - {@link FileSegmentManagedBuffer}: data backed by part of a file + * - {@link NioManagedBuffer}: data backed by a NIO ByteBuffer + * - {@link NettyManagedBuffer}: data backed by a Netty ByteBuf + * + * The concrete buffer implementation might be managed outside the JVM garbage collector. + * For example, in the case of {@link NettyManagedBuffer}, the buffers are reference counted. + * In that case, if the buffer is going to be passed around to a different thread, retain/release + * should be called. + */ +public abstract class ManagedBuffer { + + /** Number of bytes of the data. */ + public abstract long size(); + + /** + * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the + * returned ByteBuffer should not affect the content of this buffer. + */ + // TODO: Deprecate this, usage may require expensive memory mapping or allocation. + public abstract ByteBuffer nioByteBuffer() throws IOException; + + /** + * Exposes this buffer's data as an InputStream. The underlying implementation does not + * necessarily check for the length of bytes read, so the caller is responsible for making sure + * it does not go over the limit. + */ + public abstract InputStream createInputStream() throws IOException; + + /** + * Increment the reference count by one if applicable. + */ + public abstract ManagedBuffer retain(); + + /** + * If applicable, decrement the reference count by one and deallocates the buffer if the + * reference count reaches zero. + */ + public abstract ManagedBuffer release(); + + /** + * Convert the buffer into an Netty object, used to write the data out. + */ + public abstract Object convertToNetty() throws IOException; +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java new file mode 100644 index 0000000000000..c806bfa45bef3 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java @@ -0,0 +1,76 @@ +/* + * 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.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufInputStream; + +/** + * A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}. + */ +public final class NettyManagedBuffer extends ManagedBuffer { + private final ByteBuf buf; + + public NettyManagedBuffer(ByteBuf buf) { + this.buf = buf; + } + + @Override + public long size() { + return buf.readableBytes(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return buf.nioBuffer(); + } + + @Override + public InputStream createInputStream() throws IOException { + return new ByteBufInputStream(buf); + } + + @Override + public ManagedBuffer retain() { + buf.retain(); + return this; + } + + @Override + public ManagedBuffer release() { + buf.release(); + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return buf.duplicate(); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("buf", buf) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java new file mode 100644 index 0000000000000..f55b884bc45ce --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBufInputStream; +import io.netty.buffer.Unpooled; + +/** + * A {@link ManagedBuffer} backed by {@link ByteBuffer}. + */ +public final class NioManagedBuffer extends ManagedBuffer { + private final ByteBuffer buf; + + public NioManagedBuffer(ByteBuffer buf) { + this.buf = buf; + } + + @Override + public long size() { + return buf.remaining(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return buf.duplicate(); + } + + @Override + public InputStream createInputStream() throws IOException { + return new ByteBufInputStream(Unpooled.wrappedBuffer(buf)); + } + + @Override + public ManagedBuffer retain() { + return this; + } + + @Override + public ManagedBuffer release() { + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return Unpooled.wrappedBuffer(buf); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("buf", buf) + .toString(); + } +} + diff --git a/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala b/network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java similarity index 67% rename from core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala rename to network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java index 5b6d086630834..1fbdcd6780785 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala +++ b/network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java @@ -15,18 +15,17 @@ * limitations under the License. */ -package org.apache.spark.storage - -import java.nio.ByteBuffer - +package org.apache.spark.network.client; /** - * An interface for providing data for blocks. - * - * getBlockData returns either a FileSegment (for zero-copy send), or a ByteBuffer. - * - * Aside from unit tests, [[BlockManager]] is the main class that implements this. + * General exception caused by a remote exception while fetching a chunk. */ -private[spark] trait BlockDataProvider { - def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] +public class ChunkFetchFailureException extends RuntimeException { + public ChunkFetchFailureException(String errorMsg, Throwable cause) { + super(errorMsg, cause); + } + + public ChunkFetchFailureException(String errorMsg) { + super(errorMsg); + } } diff --git a/network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java b/network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java new file mode 100644 index 0000000000000..519e6cb470d0d --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java @@ -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.network.client; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * Callback for the result of a single chunk result. For a single stream, the callbacks are + * guaranteed to be called by the same thread in the same order as the requests for chunks were + * made. + * + * Note that if a general stream failure occurs, all outstanding chunk requests may be failed. + */ +public interface ChunkReceivedCallback { + /** + * Called upon receipt of a particular chunk. + * + * The given buffer will initially have a refcount of 1, but will be release()'d as soon as this + * call returns. You must therefore either retain() the buffer or copy its contents before + * returning. + */ + void onSuccess(int chunkIndex, ManagedBuffer buffer); + + /** + * Called upon failure to fetch a particular chunk. Note that this may actually be called due + * to failure to fetch a prior chunk in this stream. + * + * After receiving a failure, the stream may or may not be valid. The client should not assume + * that the server's side of the stream has been closed. + */ + void onFailure(int chunkIndex, Throwable e); +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala b/network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java similarity index 55% rename from core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala rename to network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java index 162e9cc6828d4..6ec960d795420 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala +++ b/network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java @@ -15,18 +15,16 @@ * limitations under the License. */ -package org.apache.spark.network.netty.server +package org.apache.spark.network.client; /** - * Header describing a block. This is used only in the server pipeline. - * - * [[BlockServerHandler]] creates this, and [[BlockHeaderEncoder]] encodes it. - * - * @param blockSize length of the block content, excluding the length itself. - * If positive, this is the header for a block (not part of the header). - * If negative, this is the header and content for an error message. - * @param blockId block id - * @param error some error message from reading the block + * Callback for the result of a single RPC. This will be invoked once with either success or + * failure. */ -private[server] -class BlockHeader(val blockSize: Int, val blockId: String, val error: Option[String] = None) +public interface RpcResponseCallback { + /** Successful serialized result from server. */ + void onSuccess(byte[] response); + + /** Exception either propagated from server or raised on client side. */ + void onFailure(Throwable e); +} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java new file mode 100644 index 0000000000000..b1732fcde21f1 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -0,0 +1,159 @@ +/* + * 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.network.client; + +import java.io.Closeable; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import com.google.common.base.Preconditions; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.protocol.ChunkFetchRequest; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.util.NettyUtils; + +/** + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow + * efficient transfer of a large amount of data, broken up into chunks with size ranging from + * hundreds of KB to a few MB. + * + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane), + * the actual setup of the streams is done outside the scope of the transport layer. The convenience + * method "sendRPC" is provided to enable control plane communication between the client and server + * to perform this setup. + * + * For example, a typical workflow might be: + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100 + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback) + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback) + * ... + * client.sendRPC(new CloseStream(100)) + * + * Construct an instance of TransportClient using {@link TransportClientFactory}. A single + * TransportClient may be used for multiple streams, but any given stream must be restricted to a + * single client, in order to avoid out-of-order responses. + * + * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is + * responsible for handling responses from the server. + * + * Concurrency: thread safe and can be called from multiple threads. + */ +public class TransportClient implements Closeable { + private final Logger logger = LoggerFactory.getLogger(TransportClient.class); + + private final Channel channel; + private final TransportResponseHandler handler; + + public TransportClient(Channel channel, TransportResponseHandler handler) { + this.channel = Preconditions.checkNotNull(channel); + this.handler = Preconditions.checkNotNull(handler); + } + + public boolean isActive() { + return channel.isOpen() || channel.isActive(); + } + + /** + * Requests a single chunk from the remote side, from the pre-negotiated streamId. + * + * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though + * some streams may not support this. + * + * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed + * to be returned in the same order that they were requested, assuming only a single + * TransportClient is used to fetch the chunks. + * + * @param streamId Identifier that refers to a stream in the remote StreamManager. This should + * be agreed upon by client and server beforehand. + * @param chunkIndex 0-based index of the chunk to fetch + * @param callback Callback invoked upon successful receipt of chunk, or upon any failure. + */ + public void fetchChunk( + long streamId, + final int chunkIndex, + final ChunkReceivedCallback callback) { + final String serverAddr = NettyUtils.getRemoteAddress(channel); + final long startTime = System.currentTimeMillis(); + logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr); + + final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex); + handler.addFetchRequest(streamChunkId, callback); + + channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener( + new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (future.isSuccess()) { + long timeTaken = System.currentTimeMillis() - startTime; + logger.trace("Sending request {} to {} took {} ms", streamChunkId, serverAddr, + timeTaken); + } else { + String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId, + serverAddr, future.cause()); + logger.error(errorMsg, future.cause()); + handler.removeFetchRequest(streamChunkId); + callback.onFailure(chunkIndex, new RuntimeException(errorMsg, future.cause())); + channel.close(); + } + } + }); + } + + /** + * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked + * with the server's response or upon any failure. + */ + public void sendRpc(byte[] message, final RpcResponseCallback callback) { + final String serverAddr = NettyUtils.getRemoteAddress(channel); + final long startTime = System.currentTimeMillis(); + logger.trace("Sending RPC to {}", serverAddr); + + final long requestId = UUID.randomUUID().getLeastSignificantBits(); + handler.addRpcRequest(requestId, callback); + + channel.writeAndFlush(new RpcRequest(requestId, message)).addListener( + new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (future.isSuccess()) { + long timeTaken = System.currentTimeMillis() - startTime; + logger.trace("Sending request {} to {} took {} ms", requestId, serverAddr, timeTaken); + } else { + String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId, + serverAddr, future.cause()); + logger.error(errorMsg, future.cause()); + handler.removeRpcRequest(requestId); + callback.onFailure(new RuntimeException(errorMsg, future.cause())); + channel.close(); + } + } + }); + } + + @Override + public void close() { + // close is a local operation and should finish with milliseconds; timeout just to be safe + channel.close().awaitUninterruptibly(10, TimeUnit.SECONDS); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java new file mode 100644 index 0000000000000..10eb9ef7a025f --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -0,0 +1,182 @@ +/* + * 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.network.client; + +import java.io.Closeable; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; + +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.util.internal.PlatformDependent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.server.TransportChannelHandler; +import org.apache.spark.network.util.IOMode; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Factory for creating {@link TransportClient}s by using createClient. + * + * The factory maintains a connection pool to other hosts and should return the same + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for + * all {@link TransportClient}s. + */ +public class TransportClientFactory implements Closeable { + private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); + + private final TransportContext context; + private final TransportConf conf; + private final ConcurrentHashMap connectionPool; + + private final Class socketChannelClass; + private final EventLoopGroup workerGroup; + + public TransportClientFactory(TransportContext context) { + this.context = context; + this.conf = context.getConf(); + this.connectionPool = new ConcurrentHashMap(); + + IOMode ioMode = IOMode.valueOf(conf.ioMode()); + this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); + // TODO: Make thread pool name configurable. + this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client"); + } + + /** + * Create a new BlockFetchingClient connecting to the given remote host / port. + * + * This blocks until a connection is successfully established. + * + * Concurrency: This method is safe to call from multiple threads. + */ + public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException { + // Get connection from the connection pool first. + // If it is not found or not active, create a new one. + final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); + TransportClient cachedClient = connectionPool.get(address); + if (cachedClient != null && cachedClient.isActive()) { + return cachedClient; + } else if (cachedClient != null) { + connectionPool.remove(address, cachedClient); // Remove inactive clients. + } + + logger.debug("Creating new connection to " + address); + + Bootstrap bootstrap = new Bootstrap(); + bootstrap.group(workerGroup) + .channel(socketChannelClass) + // Disable Nagle's Algorithm since we don't want packets to wait + .option(ChannelOption.TCP_NODELAY, true) + .option(ChannelOption.SO_KEEPALIVE, true) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()); + + // Use pooled buffers to reduce temporary buffer allocation + bootstrap.option(ChannelOption.ALLOCATOR, createPooledByteBufAllocator()); + + final AtomicReference client = new AtomicReference(); + + bootstrap.handler(new ChannelInitializer() { + @Override + public void initChannel(SocketChannel ch) { + TransportChannelHandler clientHandler = context.initializePipeline(ch); + client.set(clientHandler.getClient()); + } + }); + + // Connect to the remote server + ChannelFuture cf = bootstrap.connect(address); + if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { + throw new TimeoutException( + String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); + } else if (cf.cause() != null) { + throw new RuntimeException(String.format("Failed to connect to %s", address), cf.cause()); + } + + // Successful connection + assert client.get() != null : "Channel future completed successfully with null client"; + TransportClient oldClient = connectionPool.putIfAbsent(address, client.get()); + if (oldClient == null) { + return client.get(); + } else { + logger.debug("Two clients were created concurrently, second one will be disposed."); + client.get().close(); + return oldClient; + } + } + + /** Close all connections in the connection pool, and shutdown the worker thread pool. */ + @Override + public void close() { + for (TransportClient client : connectionPool.values()) { + try { + client.close(); + } catch (RuntimeException e) { + logger.warn("Ignoring exception during close", e); + } + } + connectionPool.clear(); + + if (workerGroup != null) { + workerGroup.shutdownGracefully(); + } + } + + /** + * Create a pooled ByteBuf allocator but disables the thread-local cache. Thread-local caches + * are disabled because the ByteBufs are allocated by the event loop thread, but released by the + * executor thread rather than the event loop thread. Those thread-local caches actually delay + * the recycling of buffers, leading to larger memory usage. + */ + private PooledByteBufAllocator createPooledByteBufAllocator() { + return new PooledByteBufAllocator( + PlatformDependent.directBufferPreferred(), + getPrivateStaticField("DEFAULT_NUM_HEAP_ARENA"), + getPrivateStaticField("DEFAULT_NUM_DIRECT_ARENA"), + getPrivateStaticField("DEFAULT_PAGE_SIZE"), + getPrivateStaticField("DEFAULT_MAX_ORDER"), + 0, // tinyCacheSize + 0, // smallCacheSize + 0 // normalCacheSize + ); + } + + /** Used to get defaults from Netty's private static fields. */ + private int getPrivateStaticField(String name) { + try { + Field f = PooledByteBufAllocator.DEFAULT.getClass().getDeclaredField(name); + f.setAccessible(true); + return f.getInt(null); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java new file mode 100644 index 0000000000000..d8965590b34da --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -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.network.client; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.ResponseMessage; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.server.MessageHandler; +import org.apache.spark.network.util.NettyUtils; + +/** + * Handler that processes server responses, in response to requests issued from a + * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks). + * + * Concurrency: thread safe and can be called from multiple threads. + */ +public class TransportResponseHandler extends MessageHandler { + private final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class); + + private final Channel channel; + + private final Map outstandingFetches; + + private final Map outstandingRpcs; + + public TransportResponseHandler(Channel channel) { + this.channel = channel; + this.outstandingFetches = new ConcurrentHashMap(); + this.outstandingRpcs = new ConcurrentHashMap(); + } + + public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) { + outstandingFetches.put(streamChunkId, callback); + } + + public void removeFetchRequest(StreamChunkId streamChunkId) { + outstandingFetches.remove(streamChunkId); + } + + public void addRpcRequest(long requestId, RpcResponseCallback callback) { + outstandingRpcs.put(requestId, callback); + } + + public void removeRpcRequest(long requestId) { + outstandingRpcs.remove(requestId); + } + + /** + * Fire the failure callback for all outstanding requests. This is called when we have an + * uncaught exception or pre-mature connection termination. + */ + private void failOutstandingRequests(Throwable cause) { + for (Map.Entry entry : outstandingFetches.entrySet()) { + entry.getValue().onFailure(entry.getKey().chunkIndex, cause); + } + for (Map.Entry entry : outstandingRpcs.entrySet()) { + entry.getValue().onFailure(cause); + } + + // It's OK if new fetches appear, as they will fail immediately. + outstandingFetches.clear(); + outstandingRpcs.clear(); + } + + @Override + public void channelUnregistered() { + if (numOutstandingRequests() > 0) { + String remoteAddress = NettyUtils.getRemoteAddress(channel); + logger.error("Still have {} requests outstanding when connection from {} is closed", + numOutstandingRequests(), remoteAddress); + failOutstandingRequests(new RuntimeException("Connection from " + remoteAddress + " closed")); + } + } + + @Override + public void exceptionCaught(Throwable cause) { + if (numOutstandingRequests() > 0) { + String remoteAddress = NettyUtils.getRemoteAddress(channel); + logger.error("Still have {} requests outstanding when connection from {} is closed", + numOutstandingRequests(), remoteAddress); + failOutstandingRequests(cause); + } + } + + @Override + public void handle(ResponseMessage message) { + String remoteAddress = NettyUtils.getRemoteAddress(channel); + if (message instanceof ChunkFetchSuccess) { + ChunkFetchSuccess resp = (ChunkFetchSuccess) message; + ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); + if (listener == null) { + logger.warn("Ignoring response for block {} from {} since it is not outstanding", + resp.streamChunkId, remoteAddress); + resp.buffer.release(); + } else { + outstandingFetches.remove(resp.streamChunkId); + listener.onSuccess(resp.streamChunkId.chunkIndex, resp.buffer); + resp.buffer.release(); + } + } else if (message instanceof ChunkFetchFailure) { + ChunkFetchFailure resp = (ChunkFetchFailure) message; + ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); + if (listener == null) { + logger.warn("Ignoring response for block {} from {} ({}) since it is not outstanding", + resp.streamChunkId, remoteAddress, resp.errorString); + } else { + outstandingFetches.remove(resp.streamChunkId); + listener.onFailure(resp.streamChunkId.chunkIndex, new ChunkFetchFailureException( + "Failure while fetching " + resp.streamChunkId + ": " + resp.errorString)); + } + } else if (message instanceof RpcResponse) { + RpcResponse resp = (RpcResponse) message; + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", + resp.requestId, remoteAddress, resp.response.length); + } else { + outstandingRpcs.remove(resp.requestId); + listener.onSuccess(resp.response); + } + } else if (message instanceof RpcFailure) { + RpcFailure resp = (RpcFailure) message; + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", + resp.requestId, remoteAddress, resp.errorString); + } else { + outstandingRpcs.remove(resp.requestId); + listener.onFailure(new RuntimeException(resp.errorString)); + } + } else { + throw new IllegalStateException("Unknown response type: " + message.type()); + } + } + + /** Returns total number of outstanding requests (fetch requests + rpcs) */ + @VisibleForTesting + public int numOutstandingRequests() { + return outstandingFetches.size() + outstandingRpcs.size(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java new file mode 100644 index 0000000000000..152af98ced7ce --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -0,0 +1,76 @@ +/* + * 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.network.protocol; + +import com.google.common.base.Charsets; +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** + * Response to {@link ChunkFetchRequest} when there is an error fetching the chunk. + */ +public final class ChunkFetchFailure implements ResponseMessage { + public final StreamChunkId streamChunkId; + public final String errorString; + + public ChunkFetchFailure(StreamChunkId streamChunkId, String errorString) { + this.streamChunkId = streamChunkId; + this.errorString = errorString; + } + + @Override + public Type type() { return Type.ChunkFetchFailure; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength() + 4 + errorString.getBytes(Charsets.UTF_8).length; + } + + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + byte[] errorBytes = errorString.getBytes(Charsets.UTF_8); + buf.writeInt(errorBytes.length); + buf.writeBytes(errorBytes); + } + + public static ChunkFetchFailure decode(ByteBuf buf) { + StreamChunkId streamChunkId = StreamChunkId.decode(buf); + int numErrorStringBytes = buf.readInt(); + byte[] errorBytes = new byte[numErrorStringBytes]; + buf.readBytes(errorBytes); + return new ChunkFetchFailure(streamChunkId, new String(errorBytes, Charsets.UTF_8)); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchFailure) { + ChunkFetchFailure o = (ChunkFetchFailure) other; + return streamChunkId.equals(o.streamChunkId) && errorString.equals(o.errorString); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamChunkId", streamChunkId) + .add("errorString", errorString) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java new file mode 100644 index 0000000000000..980947cf13f6b --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java @@ -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.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** + * Request to fetch a sequence of a single chunk of a stream. This will correspond to a single + * {@link org.apache.spark.network.protocol.ResponseMessage} (either success or failure). + */ +public final class ChunkFetchRequest implements RequestMessage { + public final StreamChunkId streamChunkId; + + public ChunkFetchRequest(StreamChunkId streamChunkId) { + this.streamChunkId = streamChunkId; + } + + @Override + public Type type() { return Type.ChunkFetchRequest; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength(); + } + + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + } + + public static ChunkFetchRequest decode(ByteBuf buf) { + return new ChunkFetchRequest(StreamChunkId.decode(buf)); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchRequest) { + ChunkFetchRequest o = (ChunkFetchRequest) other; + return streamChunkId.equals(o.streamChunkId); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamChunkId", streamChunkId) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java new file mode 100644 index 0000000000000..ff4936470c697 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java @@ -0,0 +1,80 @@ +/* + * 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.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; + +/** + * Response to {@link ChunkFetchRequest} when a chunk exists and has been successfully fetched. + * + * Note that the server-side encoding of this messages does NOT include the buffer itself, as this + * may be written by Netty in a more efficient manner (i.e., zero-copy write). + * Similarly, the client-side decoding will reuse the Netty ByteBuf as the buffer. + */ +public final class ChunkFetchSuccess implements ResponseMessage { + public final StreamChunkId streamChunkId; + public final ManagedBuffer buffer; + + public ChunkFetchSuccess(StreamChunkId streamChunkId, ManagedBuffer buffer) { + this.streamChunkId = streamChunkId; + this.buffer = buffer; + } + + @Override + public Type type() { return Type.ChunkFetchSuccess; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength(); + } + + /** Encoding does NOT include 'buffer' itself. See {@link MessageEncoder}. */ + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + } + + /** Decoding uses the given ByteBuf as our data, and will retain() it. */ + public static ChunkFetchSuccess decode(ByteBuf buf) { + StreamChunkId streamChunkId = StreamChunkId.decode(buf); + buf.retain(); + NettyManagedBuffer managedBuf = new NettyManagedBuffer(buf.duplicate()); + return new ChunkFetchSuccess(streamChunkId, managedBuf); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchSuccess) { + ChunkFetchSuccess o = (ChunkFetchSuccess) other; + return streamChunkId.equals(o.streamChunkId) && buffer.equals(o.buffer); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamChunkId", streamChunkId) + .add("buffer", buffer) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java b/network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java new file mode 100644 index 0000000000000..b4e299471b41a --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java @@ -0,0 +1,41 @@ +/* + * 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.network.protocol; + +import io.netty.buffer.ByteBuf; + +/** + * Interface for an object which can be encoded into a ByteBuf. Multiple Encodable objects are + * stored in a single, pre-allocated ByteBuf, so Encodables must also provide their length. + * + * Encodable objects should provide a static "decode(ByteBuf)" method which is invoked by + * {@link MessageDecoder}. During decoding, if the object uses the ByteBuf as its data (rather than + * just copying data from it), then you must retain() the ByteBuf. + * + * Additionally, when adding a new Encodable Message, add it to {@link Message.Type}. + */ +public interface Encodable { + /** Number of bytes of the encoded form of this object. */ + int encodedLength(); + + /** + * Serializes this object by writing into the given ByteBuf. + * This method must write exactly encodedLength() bytes. + */ + void encode(ByteBuf buf); +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Message.java b/network/common/src/main/java/org/apache/spark/network/protocol/Message.java new file mode 100644 index 0000000000000..d568370125fd4 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/Message.java @@ -0,0 +1,58 @@ +/* + * 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.network.protocol; + +import io.netty.buffer.ByteBuf; + +/** An on-the-wire transmittable message. */ +public interface Message extends Encodable { + /** Used to identify this request type. */ + Type type(); + + /** Preceding every serialized Message is its type, which allows us to deserialize it. */ + public static enum Type implements Encodable { + ChunkFetchRequest(0), ChunkFetchSuccess(1), ChunkFetchFailure(2), + RpcRequest(3), RpcResponse(4), RpcFailure(5); + + private final byte id; + + private Type(int id) { + assert id < 128 : "Cannot have more than 128 message types"; + this.id = (byte) id; + } + + public byte id() { return id; } + + @Override public int encodedLength() { return 1; } + + @Override public void encode(ByteBuf buf) { buf.writeByte(id); } + + public static Type decode(ByteBuf buf) { + byte id = buf.readByte(); + switch (id) { + case 0: return ChunkFetchRequest; + case 1: return ChunkFetchSuccess; + case 2: return ChunkFetchFailure; + case 3: return RpcRequest; + case 4: return RpcResponse; + case 5: return RpcFailure; + default: throw new IllegalArgumentException("Unknown message type: " + id); + } + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java new file mode 100644 index 0000000000000..81f8d7f96350f --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java @@ -0,0 +1,70 @@ +/* + * 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.network.protocol; + +import java.util.List; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToMessageDecoder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Decoder used by the client side to encode server-to-client responses. + * This encoder is stateless so it is safe to be shared by multiple threads. + */ +@ChannelHandler.Sharable +public final class MessageDecoder extends MessageToMessageDecoder { + + private final Logger logger = LoggerFactory.getLogger(MessageDecoder.class); + @Override + public void decode(ChannelHandlerContext ctx, ByteBuf in, List out) { + Message.Type msgType = Message.Type.decode(in); + Message decoded = decode(msgType, in); + assert decoded.type() == msgType; + logger.trace("Received message " + msgType + ": " + decoded); + out.add(decoded); + } + + private Message decode(Message.Type msgType, ByteBuf in) { + switch (msgType) { + case ChunkFetchRequest: + return ChunkFetchRequest.decode(in); + + case ChunkFetchSuccess: + return ChunkFetchSuccess.decode(in); + + case ChunkFetchFailure: + return ChunkFetchFailure.decode(in); + + case RpcRequest: + return RpcRequest.decode(in); + + case RpcResponse: + return RpcResponse.decode(in); + + case RpcFailure: + return RpcFailure.decode(in); + + default: + throw new IllegalArgumentException("Unexpected message type: " + msgType); + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java new file mode 100644 index 0000000000000..4cb8becc3ed22 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -0,0 +1,80 @@ +/* + * 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.network.protocol; + +import java.util.List; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToMessageEncoder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Encoder used by the server side to encode server-to-client responses. + * This encoder is stateless so it is safe to be shared by multiple threads. + */ +@ChannelHandler.Sharable +public final class MessageEncoder extends MessageToMessageEncoder { + + private final Logger logger = LoggerFactory.getLogger(MessageEncoder.class); + + /*** + * Encodes a Message by invoking its encode() method. For non-data messages, we will add one + * ByteBuf to 'out' containing the total frame length, the message type, and the message itself. + * In the case of a ChunkFetchSuccess, we will also add the ManagedBuffer corresponding to the + * data to 'out', in order to enable zero-copy transfer. + */ + @Override + public void encode(ChannelHandlerContext ctx, Message in, List out) { + Object body = null; + long bodyLength = 0; + + // Only ChunkFetchSuccesses have data besides the header. + // The body is used in order to enable zero-copy transfer for the payload. + if (in instanceof ChunkFetchSuccess) { + ChunkFetchSuccess resp = (ChunkFetchSuccess) in; + try { + bodyLength = resp.buffer.size(); + body = resp.buffer.convertToNetty(); + } catch (Exception e) { + // Re-encode this message as BlockFetchFailure. + logger.error(String.format("Error opening block %s for client %s", + resp.streamChunkId, ctx.channel().remoteAddress()), e); + encode(ctx, new ChunkFetchFailure(resp.streamChunkId, e.getMessage()), out); + return; + } + } + + Message.Type msgType = in.type(); + // All messages have the frame length, message type, and message itself. + int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); + long frameLength = headerLength + bodyLength; + ByteBuf header = ctx.alloc().buffer(headerLength); + header.writeLong(frameLength); + msgType.encode(header); + in.encode(header); + assert header.writableBytes() == 0; + + out.add(header); + if (body != null && bodyLength > 0) { + out.add(body); + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala b/network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java similarity index 77% rename from core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala rename to network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java index 0d7695072a7b1..31b15bb17a327 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.network.netty +package org.apache.spark.network.protocol; -import org.apache.spark.storage.{BlockId, FileSegment} +import org.apache.spark.network.protocol.Message; -trait PathResolver { - /** Get the file segment in which the given block resides. */ - def getBlockLocation(blockId: BlockId): FileSegment +/** Messages from the client to the server. */ +public interface RequestMessage extends Message { + // token interface } diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala b/network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java similarity index 75% rename from core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala rename to network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java index e28219dd7745b..6edffd11cf1e2 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java @@ -15,15 +15,11 @@ * limitations under the License. */ -package org.apache.spark.network.netty.client +package org.apache.spark.network.protocol; -import java.util.EventListener - - -trait BlockClientListener extends EventListener { - - def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit - - def onFetchFailure(blockId: String, errorMsg: String): Unit +import org.apache.spark.network.protocol.Message; +/** Messages from the server to the client. */ +public interface ResponseMessage extends Message { + // token interface } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java new file mode 100644 index 0000000000000..e239d4ffbd29c --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -0,0 +1,74 @@ +/* + * 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.network.protocol; + +import com.google.common.base.Charsets; +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** Response to {@link RpcRequest} for a failed RPC. */ +public final class RpcFailure implements ResponseMessage { + public final long requestId; + public final String errorString; + + public RpcFailure(long requestId, String errorString) { + this.requestId = requestId; + this.errorString = errorString; + } + + @Override + public Type type() { return Type.RpcFailure; } + + @Override + public int encodedLength() { + return 8 + 4 + errorString.getBytes(Charsets.UTF_8).length; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + byte[] errorBytes = errorString.getBytes(Charsets.UTF_8); + buf.writeInt(errorBytes.length); + buf.writeBytes(errorBytes); + } + + public static RpcFailure decode(ByteBuf buf) { + long requestId = buf.readLong(); + int numErrorStringBytes = buf.readInt(); + byte[] errorBytes = new byte[numErrorStringBytes]; + buf.readBytes(errorBytes); + return new RpcFailure(requestId, new String(errorBytes, Charsets.UTF_8)); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcFailure) { + RpcFailure o = (RpcFailure) other; + return requestId == o.requestId && errorString.equals(o.errorString); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("errorString", errorString) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java new file mode 100644 index 0000000000000..099e934ae018c --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java @@ -0,0 +1,81 @@ +/* + * 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.network.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** + * A generic RPC which is handled by a remote {@link org.apache.spark.network.server.RpcHandler}. + * This will correspond to a single + * {@link org.apache.spark.network.protocol.ResponseMessage} (either success or failure). + */ +public final class RpcRequest implements RequestMessage { + /** Used to link an RPC request with its response. */ + public final long requestId; + + /** Serialized message to send to remote RpcHandler. */ + public final byte[] message; + + public RpcRequest(long requestId, byte[] message) { + this.requestId = requestId; + this.message = message; + } + + @Override + public Type type() { return Type.RpcRequest; } + + @Override + public int encodedLength() { + return 8 + 4 + message.length; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + buf.writeInt(message.length); + buf.writeBytes(message); + } + + public static RpcRequest decode(ByteBuf buf) { + long requestId = buf.readLong(); + int messageLen = buf.readInt(); + byte[] message = new byte[messageLen]; + buf.readBytes(message); + return new RpcRequest(requestId, message); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcRequest) { + RpcRequest o = (RpcRequest) other; + return requestId == o.requestId && Arrays.equals(message, o.message); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("message", message) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java new file mode 100644 index 0000000000000..ed479478325b6 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java @@ -0,0 +1,72 @@ +/* + * 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.network.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** Response to {@link RpcRequest} for a successful RPC. */ +public final class RpcResponse implements ResponseMessage { + public final long requestId; + public final byte[] response; + + public RpcResponse(long requestId, byte[] response) { + this.requestId = requestId; + this.response = response; + } + + @Override + public Type type() { return Type.RpcResponse; } + + @Override + public int encodedLength() { return 8 + 4 + response.length; } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + buf.writeInt(response.length); + buf.writeBytes(response); + } + + public static RpcResponse decode(ByteBuf buf) { + long requestId = buf.readLong(); + int responseLen = buf.readInt(); + byte[] response = new byte[responseLen]; + buf.readBytes(response); + return new RpcResponse(requestId, response); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcResponse) { + RpcResponse o = (RpcResponse) other; + return requestId == o.requestId && Arrays.equals(response, o.response); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("response", response) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java b/network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java new file mode 100644 index 0000000000000..d46a263884807 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** +* Encapsulates a request for a particular chunk of a stream. +*/ +public final class StreamChunkId implements Encodable { + public final long streamId; + public final int chunkIndex; + + public StreamChunkId(long streamId, int chunkIndex) { + this.streamId = streamId; + this.chunkIndex = chunkIndex; + } + + @Override + public int encodedLength() { + return 8 + 4; + } + + public void encode(ByteBuf buffer) { + buffer.writeLong(streamId); + buffer.writeInt(chunkIndex); + } + + public static StreamChunkId decode(ByteBuf buffer) { + assert buffer.readableBytes() >= 8 + 4; + long streamId = buffer.readLong(); + int chunkIndex = buffer.readInt(); + return new StreamChunkId(streamId, chunkIndex); + } + + @Override + public int hashCode() { + return Objects.hashCode(streamId, chunkIndex); + } + + @Override + public boolean equals(Object other) { + if (other instanceof StreamChunkId) { + StreamChunkId o = (StreamChunkId) other; + return streamId == o.streamId && chunkIndex == o.chunkIndex; + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamId", streamId) + .add("chunkIndex", chunkIndex) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java new file mode 100644 index 0000000000000..9688705569634 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.util.Iterator; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * StreamManager which allows registration of an Iterator, which are individually + * fetched as chunks by the client. + */ +public class DefaultStreamManager extends StreamManager { + private final Logger logger = LoggerFactory.getLogger(DefaultStreamManager.class); + + private final AtomicLong nextStreamId; + private final Map streams; + + /** State of a single stream. */ + private static class StreamState { + final Iterator buffers; + + // Used to keep track of the index of the buffer that the user has retrieved, just to ensure + // that the caller only requests each chunk one at a time, in order. + int curChunk = 0; + + StreamState(Iterator buffers) { + this.buffers = buffers; + } + } + + public DefaultStreamManager() { + // For debugging purposes, start with a random stream id to help identifying different streams. + // This does not need to be globally unique, only unique to this class. + nextStreamId = new AtomicLong((long) new Random().nextInt(Integer.MAX_VALUE) * 1000); + streams = new ConcurrentHashMap(); + } + + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + StreamState state = streams.get(streamId); + if (chunkIndex != state.curChunk) { + throw new IllegalStateException(String.format( + "Received out-of-order chunk index %s (expected %s)", chunkIndex, state.curChunk)); + } else if (!state.buffers.hasNext()) { + throw new IllegalStateException(String.format( + "Requested chunk index beyond end %s", chunkIndex)); + } + state.curChunk += 1; + ManagedBuffer nextChunk = state.buffers.next(); + + if (!state.buffers.hasNext()) { + logger.trace("Removing stream id {}", streamId); + streams.remove(streamId); + } + + return nextChunk; + } + + @Override + public void connectionTerminated(long streamId) { + // Release all remaining buffers. + StreamState state = streams.remove(streamId); + if (state != null && state.buffers != null) { + while (state.buffers.hasNext()) { + state.buffers.next().release(); + } + } + } + + /** + * Registers a stream of ManagedBuffers which are served as individual chunks one at a time to + * callers. Each ManagedBuffer will be release()'d after it is transferred on the wire. If a + * client connection is closed before the iterator is fully drained, then the remaining buffers + * will all be release()'d. + */ + public long registerStream(Iterator buffers) { + long myStreamId = nextStreamId.getAndIncrement(); + streams.put(myStreamId, new StreamState(buffers)); + return myStreamId; + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java b/network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java new file mode 100644 index 0000000000000..b80c15106ecbd --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java @@ -0,0 +1,36 @@ +/* + * 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.network.server; + +import org.apache.spark.network.protocol.Message; + +/** + * Handles either request or response messages coming off of Netty. A MessageHandler instance + * is associated with a single Netty Channel (though it may have multiple clients on the same + * Channel.) + */ +public abstract class MessageHandler { + /** Handles the receipt of a single message. */ + public abstract void handle(T message); + + /** Invoked when an exception was caught on the Channel. */ + public abstract void exceptionCaught(Throwable cause); + + /** Invoked when the channel this MessageHandler is on has been unregistered. */ + public abstract void channelUnregistered(); +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java new file mode 100644 index 0000000000000..f54a696b8ff79 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; + +/** + * Handler for sendRPC() messages sent by {@link org.apache.spark.network.client.TransportClient}s. + */ +public interface RpcHandler { + /** + * Receive a single RPC message. Any exception thrown while in this method will be sent back to + * the client in string form as a standard RPC failure. + * + * @param client A channel client which enables the handler to make requests back to the sender + * of this RPC. + * @param message The serialized bytes of the RPC. + * @param callback Callback which should be invoked exactly once upon success or failure of the + * RPC. + */ + void receive(TransportClient client, byte[] message, RpcResponseCallback callback); +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java new file mode 100644 index 0000000000000..5a9a14a180c10 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java @@ -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.network.server; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * The StreamManager is used to fetch individual chunks from a stream. This is used in + * {@link TransportRequestHandler} in order to respond to fetchChunk() requests. Creation of the + * stream is outside the scope of the transport layer, but a given stream is guaranteed to be read + * by only one client connection, meaning that getChunk() for a particular stream will be called + * serially and that once the connection associated with the stream is closed, that stream will + * never be used again. + */ +public abstract class StreamManager { + /** + * Called in response to a fetchChunk() request. The returned buffer will be passed as-is to the + * client. A single stream will be associated with a single TCP connection, so this method + * will not be called in parallel for a particular stream. + * + * Chunks may be requested in any order, and requests may be repeated, but it is not required + * that implementations support this behavior. + * + * The returned ManagedBuffer will be release()'d after being written to the network. + * + * @param streamId id of a stream that has been previously registered with the StreamManager. + * @param chunkIndex 0-indexed chunk of the stream that's requested + */ + public abstract ManagedBuffer getChunk(long streamId, int chunkIndex); + + /** + * Indicates that the TCP connection that was tied to the given stream has been terminated. After + * this occurs, we are guaranteed not to read from the stream again, so any state can be cleaned + * up. + */ + public void connectionTerminated(long streamId) { } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java new file mode 100644 index 0000000000000..e491367fa4528 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -0,0 +1,96 @@ +/* + * 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.network.server; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.Message; +import org.apache.spark.network.protocol.RequestMessage; +import org.apache.spark.network.protocol.ResponseMessage; +import org.apache.spark.network.util.NettyUtils; + +/** + * The single Transport-level Channel handler which is used for delegating requests to the + * {@link TransportRequestHandler} and responses to the {@link TransportResponseHandler}. + * + * All channels created in the transport layer are bidirectional. When the Client initiates a Netty + * Channel with a RequestMessage (which gets handled by the Server's RequestHandler), the Server + * will produce a ResponseMessage (handled by the Client's ResponseHandler). However, the Server + * also gets a handle on the same Channel, so it may then begin to send RequestMessages to the + * Client. + * This means that the Client also needs a RequestHandler and the Server needs a ResponseHandler, + * for the Client's responses to the Server's requests. + */ +public class TransportChannelHandler extends SimpleChannelInboundHandler { + private final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class); + + private final TransportClient client; + private final TransportResponseHandler responseHandler; + private final TransportRequestHandler requestHandler; + + public TransportChannelHandler( + TransportClient client, + TransportResponseHandler responseHandler, + TransportRequestHandler requestHandler) { + this.client = client; + this.responseHandler = responseHandler; + this.requestHandler = requestHandler; + } + + public TransportClient getClient() { + return client; + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + logger.warn("Exception in connection from " + NettyUtils.getRemoteAddress(ctx.channel()), + cause); + requestHandler.exceptionCaught(cause); + responseHandler.exceptionCaught(cause); + ctx.close(); + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + try { + requestHandler.channelUnregistered(); + } catch (RuntimeException e) { + logger.error("Exception from request handler while unregistering channel", e); + } + try { + responseHandler.channelUnregistered(); + } catch (RuntimeException e) { + logger.error("Exception from response handler while unregistering channel", e); + } + super.channelUnregistered(ctx); + } + + @Override + public void channelRead0(ChannelHandlerContext ctx, Message request) { + if (request instanceof RequestMessage) { + requestHandler.handle((RequestMessage) request); + } else { + responseHandler.handle((ResponseMessage) request); + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java new file mode 100644 index 0000000000000..352f865935b11 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.util.Set; + +import com.google.common.base.Throwables; +import com.google.common.collect.Sets; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.RequestMessage; +import org.apache.spark.network.protocol.ChunkFetchRequest; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.util.NettyUtils; + +/** + * A handler that processes requests from clients and writes chunk data back. Each handler is + * attached to a single Netty channel, and keeps track of which streams have been fetched via this + * channel, in order to clean them up if the channel is terminated (see #channelUnregistered). + * + * The messages should have been processed by the pipeline setup by {@link TransportServer}. + */ +public class TransportRequestHandler extends MessageHandler { + private final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class); + + /** The Netty channel that this handler is associated with. */ + private final Channel channel; + + /** Client on the same channel allowing us to talk back to the requester. */ + private final TransportClient reverseClient; + + /** Returns each chunk part of a stream. */ + private final StreamManager streamManager; + + /** Handles all RPC messages. */ + private final RpcHandler rpcHandler; + + /** List of all stream ids that have been read on this handler, used for cleanup. */ + private final Set streamIds; + + public TransportRequestHandler( + Channel channel, + TransportClient reverseClient, + StreamManager streamManager, + RpcHandler rpcHandler) { + this.channel = channel; + this.reverseClient = reverseClient; + this.streamManager = streamManager; + this.rpcHandler = rpcHandler; + this.streamIds = Sets.newHashSet(); + } + + @Override + public void exceptionCaught(Throwable cause) { + } + + @Override + public void channelUnregistered() { + // Inform the StreamManager that these streams will no longer be read from. + for (long streamId : streamIds) { + streamManager.connectionTerminated(streamId); + } + } + + @Override + public void handle(RequestMessage request) { + if (request instanceof ChunkFetchRequest) { + processFetchRequest((ChunkFetchRequest) request); + } else if (request instanceof RpcRequest) { + processRpcRequest((RpcRequest) request); + } else { + throw new IllegalArgumentException("Unknown request type: " + request); + } + } + + private void processFetchRequest(final ChunkFetchRequest req) { + final String client = NettyUtils.getRemoteAddress(channel); + streamIds.add(req.streamChunkId.streamId); + + logger.trace("Received req from {} to fetch block {}", client, req.streamChunkId); + + ManagedBuffer buf; + try { + buf = streamManager.getChunk(req.streamChunkId.streamId, req.streamChunkId.chunkIndex); + } catch (Exception e) { + logger.error(String.format( + "Error opening block %s for request from %s", req.streamChunkId, client), e); + respond(new ChunkFetchFailure(req.streamChunkId, Throwables.getStackTraceAsString(e))); + return; + } + + respond(new ChunkFetchSuccess(req.streamChunkId, buf)); + } + + private void processRpcRequest(final RpcRequest req) { + try { + rpcHandler.receive(reverseClient, req.message, new RpcResponseCallback() { + @Override + public void onSuccess(byte[] response) { + respond(new RpcResponse(req.requestId, response)); + } + + @Override + public void onFailure(Throwable e) { + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + }); + } catch (Exception e) { + logger.error("Error while invoking RpcHandler#receive() on RPC id " + req.requestId, e); + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + } + + /** + * Responds to a single message with some Encodable object. If a failure occurs while sending, + * it will be logged and the channel closed. + */ + private void respond(final Encodable result) { + final String remoteAddress = channel.remoteAddress().toString(); + channel.writeAndFlush(result).addListener( + new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (future.isSuccess()) { + logger.trace(String.format("Sent result %s to client %s", result, remoteAddress)); + } else { + logger.error(String.format("Error sending result %s to %s; closing connection", + result, remoteAddress), future.cause()); + channel.close(); + } + } + } + ); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java new file mode 100644 index 0000000000000..243070750d6e7 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -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.network.server; + +import java.io.Closeable; +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.util.IOMode; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Server for the efficient, low-level streaming service. + */ +public class TransportServer implements Closeable { + private final Logger logger = LoggerFactory.getLogger(TransportServer.class); + + private final TransportContext context; + private final TransportConf conf; + + private ServerBootstrap bootstrap; + private ChannelFuture channelFuture; + private int port = -1; + + public TransportServer(TransportContext context) { + this.context = context; + this.conf = context.getConf(); + + init(); + } + + public int getPort() { + if (port == -1) { + throw new IllegalStateException("Server not initialized"); + } + return port; + } + + private void init() { + + IOMode ioMode = IOMode.valueOf(conf.ioMode()); + EventLoopGroup bossGroup = + NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); + EventLoopGroup workerGroup = bossGroup; + + bootstrap = new ServerBootstrap() + .group(bossGroup, workerGroup) + .channel(NettyUtils.getServerChannelClass(ioMode)) + .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT); + + if (conf.backLog() > 0) { + bootstrap.option(ChannelOption.SO_BACKLOG, conf.backLog()); + } + + if (conf.receiveBuf() > 0) { + bootstrap.childOption(ChannelOption.SO_RCVBUF, conf.receiveBuf()); + } + + if (conf.sendBuf() > 0) { + bootstrap.childOption(ChannelOption.SO_SNDBUF, conf.sendBuf()); + } + + bootstrap.childHandler(new ChannelInitializer() { + @Override + protected void initChannel(SocketChannel ch) throws Exception { + context.initializePipeline(ch); + } + }); + + channelFuture = bootstrap.bind(new InetSocketAddress(conf.serverPort())); + channelFuture.syncUninterruptibly(); + + port = ((InetSocketAddress) channelFuture.channel().localAddress()).getPort(); + logger.debug("Shuffle server started on port :" + port); + } + + @Override + public void close() { + if (channelFuture != null) { + // close is a local operation and should finish with milliseconds; timeout just to be safe + channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS); + channelFuture = null; + } + if (bootstrap != null && bootstrap.group() != null) { + bootstrap.group().shutdownGracefully(); + } + if (bootstrap != null && bootstrap.childGroup() != null) { + bootstrap.childGroup().shutdownGracefully(); + } + bootstrap = null; + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java b/network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java new file mode 100644 index 0000000000000..d944d9da1c7f8 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java @@ -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.network.util; + +import java.util.NoSuchElementException; + +/** + * Provides a mechanism for constructing a {@link TransportConf} using some sort of configuration. + */ +public abstract class ConfigProvider { + /** Obtains the value of the given config, throws NoSuchElementException if it doesn't exist. */ + public abstract String get(String name); + + public String get(String name, String defaultValue) { + try { + return get(name); + } catch (NoSuchElementException e) { + return defaultValue; + } + } + + public int getInt(String name, int defaultValue) { + return Integer.parseInt(get(name, Integer.toString(defaultValue))); + } + + public long getLong(String name, long defaultValue) { + return Long.parseLong(get(name, Long.toString(defaultValue))); + } + + public double getDouble(String name, double defaultValue) { + return Double.parseDouble(get(name, Double.toString(defaultValue))); + } + + public boolean getBoolean(String name, boolean defaultValue) { + return Boolean.parseBoolean(get(name, Boolean.toString(defaultValue))); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/IOMode.java b/network/common/src/main/java/org/apache/spark/network/util/IOMode.java new file mode 100644 index 0000000000000..6b208d95bbfbc --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/IOMode.java @@ -0,0 +1,27 @@ +/* + * 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.network.util; + +/** + * Selector for which form of low-level IO we should use. + * NIO is always available, while EPOLL is only available on Linux. + * AUTO is used to select EPOLL if it's available, or NIO otherwise. + */ +public enum IOMode { + NIO, EPOLL +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java new file mode 100644 index 0000000000000..32ba3f5b07f7a --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.io.Closeable; +import java.io.IOException; + +import com.google.common.io.Closeables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JavaUtils { + private static final Logger logger = LoggerFactory.getLogger(JavaUtils.class); + + /** Closes the given object, ignoring IOExceptions. */ + public static void closeQuietly(Closeable closeable) { + try { + closeable.close(); + } catch (IOException e) { + logger.error("IOException should not have been thrown.", e); + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java new file mode 100644 index 0000000000000..b1872341198e0 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.util.concurrent.ThreadFactory; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.netty.channel.Channel; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.ServerChannel; +import io.netty.channel.epoll.Epoll; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.epoll.EpollServerSocketChannel; +import io.netty.channel.epoll.EpollSocketChannel; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; + +/** + * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO. + */ +public class NettyUtils { + /** Creates a Netty EventLoopGroup based on the IOMode. */ + public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, String threadPrefix) { + + ThreadFactory threadFactory = new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(threadPrefix + "-%d") + .build(); + + switch (mode) { + case NIO: + return new NioEventLoopGroup(numThreads, threadFactory); + case EPOLL: + return new EpollEventLoopGroup(numThreads, threadFactory); + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** Returns the correct (client) SocketChannel class based on IOMode. */ + public static Class getClientChannelClass(IOMode mode) { + switch (mode) { + case NIO: + return NioSocketChannel.class; + case EPOLL: + return EpollSocketChannel.class; + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** Returns the correct ServerSocketChannel class based on IOMode. */ + public static Class getServerChannelClass(IOMode mode) { + switch (mode) { + case NIO: + return NioServerSocketChannel.class; + case EPOLL: + return EpollServerSocketChannel.class; + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** + * Creates a LengthFieldBasedFrameDecoder where the first 8 bytes are the length of the frame. + * This is used before all decoders. + */ + public static ByteToMessageDecoder createFrameDecoder() { + // maxFrameLength = 2G + // lengthFieldOffset = 0 + // lengthFieldLength = 8 + // lengthAdjustment = -8, i.e. exclude the 8 byte length itself + // initialBytesToStrip = 8, i.e. strip out the length field itself + return new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 8, -8, 8); + } + + /** Returns the remote address on the channel or "" if none exists. */ + public static String getRemoteAddress(Channel channel) { + if (channel != null && channel.remoteAddress() != null) { + return channel.remoteAddress().toString(); + } + return ""; + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java new file mode 100644 index 0000000000000..80f65d98032da --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -0,0 +1,61 @@ +/* + * 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.network.util; + +/** + * A central location that tracks all the settings we expose to users. + */ +public class TransportConf { + private final ConfigProvider conf; + + public TransportConf(ConfigProvider conf) { + this.conf = conf; + } + + /** Port the server listens on. Default to a random port. */ + public int serverPort() { return conf.getInt("spark.shuffle.io.port", 0); } + + /** IO mode: nio or epoll */ + public String ioMode() { return conf.get("spark.shuffle.io.mode", "NIO").toUpperCase(); } + + /** Connect timeout in secs. Default 120 secs. */ + public int connectionTimeoutMs() { + return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; + } + + /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ + public int backLog() { return conf.getInt("spark.shuffle.io.backLog", -1); } + + /** Number of threads used in the server thread pool. Default to 0, which is 2x#cores. */ + public int serverThreads() { return conf.getInt("spark.shuffle.io.serverThreads", 0); } + + /** Number of threads used in the client thread pool. Default to 0, which is 2x#cores. */ + public int clientThreads() { return conf.getInt("spark.shuffle.io.clientThreads", 0); } + + /** + * Receive buffer size (SO_RCVBUF). + * Note: the optimal size for receive buffer and send buffer should be + * latency * network_bandwidth. + * Assuming latency = 1ms, network_bandwidth = 10Gbps + * buffer size should be ~ 1.25MB + */ + public int receiveBuf() { return conf.getInt("spark.shuffle.io.receiveBuffer", -1); } + + /** Send buffer size (SO_SNDBUF). */ + public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } +} diff --git a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java new file mode 100644 index 0000000000000..738dca9b6a9ee --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -0,0 +1,217 @@ +/* + * 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.network; + +import java.io.File; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.TransportConf; + +public class ChunkFetchIntegrationSuite { + static final long STREAM_ID = 1; + static final int BUFFER_CHUNK_INDEX = 0; + static final int FILE_CHUNK_INDEX = 1; + + static TransportServer server; + static TransportClientFactory clientFactory; + static StreamManager streamManager; + static File testFile; + + static ManagedBuffer bufferChunk; + static ManagedBuffer fileChunk; + + @BeforeClass + public static void setUp() throws Exception { + int bufSize = 100000; + final ByteBuffer buf = ByteBuffer.allocate(bufSize); + for (int i = 0; i < bufSize; i ++) { + buf.put((byte) i); + } + buf.flip(); + bufferChunk = new NioManagedBuffer(buf); + + testFile = File.createTempFile("shuffle-test-file", "txt"); + testFile.deleteOnExit(); + RandomAccessFile fp = new RandomAccessFile(testFile, "rw"); + byte[] fileContent = new byte[1024]; + new Random().nextBytes(fileContent); + fp.write(fileContent); + fp.close(); + fileChunk = new FileSegmentManagedBuffer(testFile, 10, testFile.length() - 25); + + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + streamManager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + assertEquals(STREAM_ID, streamId); + if (chunkIndex == BUFFER_CHUNK_INDEX) { + return new NioManagedBuffer(buf); + } else if (chunkIndex == FILE_CHUNK_INDEX) { + return new FileSegmentManagedBuffer(testFile, 10, testFile.length() - 25); + } else { + throw new IllegalArgumentException("Invalid chunk index: " + chunkIndex); + } + } + }; + TransportContext context = new TransportContext(conf, streamManager, new NoOpRpcHandler()); + server = context.createServer(); + clientFactory = context.createClientFactory(); + } + + @AfterClass + public static void tearDown() { + server.close(); + clientFactory.close(); + testFile.delete(); + } + + class FetchResult { + public Set successChunks; + public Set failedChunks; + public List buffers; + + public void releaseBuffers() { + for (ManagedBuffer buffer : buffers) { + buffer.release(); + } + } + } + + private FetchResult fetchChunks(List chunkIndices) throws Exception { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + final Semaphore sem = new Semaphore(0); + + final FetchResult res = new FetchResult(); + res.successChunks = Collections.synchronizedSet(new HashSet()); + res.failedChunks = Collections.synchronizedSet(new HashSet()); + res.buffers = Collections.synchronizedList(new LinkedList()); + + ChunkReceivedCallback callback = new ChunkReceivedCallback() { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + buffer.retain(); + res.successChunks.add(chunkIndex); + res.buffers.add(buffer); + sem.release(); + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + res.failedChunks.add(chunkIndex); + sem.release(); + } + }; + + for (int chunkIndex : chunkIndices) { + client.fetchChunk(STREAM_ID, chunkIndex, callback); + } + if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + client.close(); + return res; + } + + @Test + public void fetchBufferChunk() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX)); + assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX)); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk)); + res.releaseBuffers(); + } + + @Test + public void fetchFileChunk() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(FILE_CHUNK_INDEX)); + assertEquals(res.successChunks, Sets.newHashSet(FILE_CHUNK_INDEX)); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(res.buffers, Lists.newArrayList(fileChunk)); + res.releaseBuffers(); + } + + @Test + public void fetchNonExistentChunk() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(12345)); + assertTrue(res.successChunks.isEmpty()); + assertEquals(res.failedChunks, Sets.newHashSet(12345)); + assertTrue(res.buffers.isEmpty()); + } + + @Test + public void fetchBothChunks() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk, fileChunk)); + res.releaseBuffers(); + } + + @Test + public void fetchChunkAndNonExistent() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX, 12345)); + assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX)); + assertEquals(res.failedChunks, Sets.newHashSet(12345)); + assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk)); + res.releaseBuffers(); + } + + private void assertBufferListsEqual(List list0, List list1) + throws Exception { + assertEquals(list0.size(), list1.size()); + for (int i = 0; i < list0.size(); i ++) { + assertBuffersEqual(list0.get(i), list1.get(i)); + } + } + + private void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) throws Exception { + ByteBuffer nio0 = buffer0.nioByteBuffer(); + ByteBuffer nio1 = buffer1.nioByteBuffer(); + + int len = nio0.remaining(); + assertEquals(nio0.remaining(), nio1.remaining()); + for (int i = 0; i < len; i ++) { + assertEquals(nio0.get(), nio1.get()); + } + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java b/network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java new file mode 100644 index 0000000000000..7aa37efc582e4 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java @@ -0,0 +1,28 @@ +package org.apache.spark.network;/* + * 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. + */ + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.server.RpcHandler; + +/** Test RpcHandler which always returns a zero-sized success. */ +public class NoOpRpcHandler implements RpcHandler { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + callback.onSuccess(new byte[0]); + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java b/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java new file mode 100644 index 0000000000000..43dc0cf8c7194 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java @@ -0,0 +1,86 @@ +/* + * 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.network; + +import io.netty.channel.embedded.EmbeddedChannel; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +import org.apache.spark.network.protocol.Message; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.protocol.ChunkFetchRequest; +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.MessageDecoder; +import org.apache.spark.network.protocol.MessageEncoder; +import org.apache.spark.network.util.NettyUtils; + +public class ProtocolSuite { + private void testServerToClient(Message msg) { + EmbeddedChannel serverChannel = new EmbeddedChannel(new MessageEncoder()); + serverChannel.writeOutbound(msg); + + EmbeddedChannel clientChannel = new EmbeddedChannel( + NettyUtils.createFrameDecoder(), new MessageDecoder()); + + while (!serverChannel.outboundMessages().isEmpty()) { + clientChannel.writeInbound(serverChannel.readOutbound()); + } + + assertEquals(1, clientChannel.inboundMessages().size()); + assertEquals(msg, clientChannel.readInbound()); + } + + private void testClientToServer(Message msg) { + EmbeddedChannel clientChannel = new EmbeddedChannel(new MessageEncoder()); + clientChannel.writeOutbound(msg); + + EmbeddedChannel serverChannel = new EmbeddedChannel( + NettyUtils.createFrameDecoder(), new MessageDecoder()); + + while (!clientChannel.outboundMessages().isEmpty()) { + serverChannel.writeInbound(clientChannel.readOutbound()); + } + + assertEquals(1, serverChannel.inboundMessages().size()); + assertEquals(msg, serverChannel.readInbound()); + } + + @Test + public void requests() { + testClientToServer(new ChunkFetchRequest(new StreamChunkId(1, 2))); + testClientToServer(new RpcRequest(12345, new byte[0])); + testClientToServer(new RpcRequest(12345, new byte[100])); + } + + @Test + public void responses() { + testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(10))); + testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(0))); + testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), "this is an error")); + testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), "")); + testServerToClient(new RpcResponse(12345, new byte[0])); + testServerToClient(new RpcResponse(12345, new byte[1000])); + testServerToClient(new RpcFailure(0, "this is an error")); + testServerToClient(new RpcFailure(0, "")); + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java new file mode 100644 index 0000000000000..9f216dd2d722d --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -0,0 +1,175 @@ +/* + * 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.network; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import com.google.common.base.Charsets; +import com.google.common.collect.Sets; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.DefaultStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.TransportConf; + +public class RpcIntegrationSuite { + static TransportServer server; + static TransportClientFactory clientFactory; + static RpcHandler rpcHandler; + + @BeforeClass + public static void setUp() throws Exception { + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + rpcHandler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + String msg = new String(message, Charsets.UTF_8); + String[] parts = msg.split("/"); + if (parts[0].equals("hello")) { + callback.onSuccess(("Hello, " + parts[1] + "!").getBytes(Charsets.UTF_8)); + } else if (parts[0].equals("return error")) { + callback.onFailure(new RuntimeException("Returned: " + parts[1])); + } else if (parts[0].equals("throw error")) { + throw new RuntimeException("Thrown: " + parts[1]); + } + } + }; + TransportContext context = new TransportContext(conf, new DefaultStreamManager(), rpcHandler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + } + + @AfterClass + public static void tearDown() { + server.close(); + clientFactory.close(); + } + + class RpcResult { + public Set successMessages; + public Set errorMessages; + } + + private RpcResult sendRPC(String ... commands) throws Exception { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + final Semaphore sem = new Semaphore(0); + + final RpcResult res = new RpcResult(); + res.successMessages = Collections.synchronizedSet(new HashSet()); + res.errorMessages = Collections.synchronizedSet(new HashSet()); + + RpcResponseCallback callback = new RpcResponseCallback() { + @Override + public void onSuccess(byte[] message) { + res.successMessages.add(new String(message, Charsets.UTF_8)); + sem.release(); + } + + @Override + public void onFailure(Throwable e) { + res.errorMessages.add(e.getMessage()); + sem.release(); + } + }; + + for (String command : commands) { + client.sendRpc(command.getBytes(Charsets.UTF_8), callback); + } + + if (!sem.tryAcquire(commands.length, 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + client.close(); + return res; + } + + @Test + public void singleRPC() throws Exception { + RpcResult res = sendRPC("hello/Aaron"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Aaron!")); + assertTrue(res.errorMessages.isEmpty()); + } + + @Test + public void doubleRPC() throws Exception { + RpcResult res = sendRPC("hello/Aaron", "hello/Reynold"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Aaron!", "Hello, Reynold!")); + assertTrue(res.errorMessages.isEmpty()); + } + + @Test + public void returnErrorRPC() throws Exception { + RpcResult res = sendRPC("return error/OK"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Returned: OK")); + } + + @Test + public void throwErrorRPC() throws Exception { + RpcResult res = sendRPC("throw error/uh-oh"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: uh-oh")); + } + + @Test + public void doubleTrouble() throws Exception { + RpcResult res = sendRPC("return error/OK", "throw error/uh-oh"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Returned: OK", "Thrown: uh-oh")); + } + + @Test + public void sendSuccessAndFailure() throws Exception { + RpcResult res = sendRPC("hello/Bob", "throw error/the", "hello/Builder", "return error/!"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Bob!", "Hello, Builder!")); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: the", "Returned: !")); + } + + private void assertErrorsContain(Set errors, Set contains) { + assertEquals(contains.size(), errors.size()); + + Set remainingErrors = Sets.newHashSet(errors); + for (String contain : contains) { + Iterator it = remainingErrors.iterator(); + boolean foundMatch = false; + while (it.hasNext()) { + if (it.next().contains(contain)) { + it.remove(); + foundMatch = true; + break; + } + } + assertTrue("Could not find error containing " + contain + "; errors: " + errors, foundMatch); + } + + assertTrue(remainingErrors.isEmpty()); + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java b/network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java new file mode 100644 index 0000000000000..f4e0a2426a3d2 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java @@ -0,0 +1,34 @@ +/* + * 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.network; + +import java.util.NoSuchElementException; + +import org.apache.spark.network.util.ConfigProvider; + +/** Uses System properties to obtain config values. */ +public class SystemPropertyConfigProvider extends ConfigProvider { + @Override + public String get(String name) { + String value = System.getProperty(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java new file mode 100644 index 0000000000000..38113a918f795 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import com.google.common.base.Preconditions; +import io.netty.buffer.Unpooled; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; + +/** + * A ManagedBuffer implementation that contains 0, 1, 2, 3, ..., (len-1). + * + * Used for testing. + */ +public class TestManagedBuffer extends ManagedBuffer { + + private final int len; + private NettyManagedBuffer underlying; + + public TestManagedBuffer(int len) { + Preconditions.checkArgument(len <= Byte.MAX_VALUE); + this.len = len; + byte[] byteArray = new byte[len]; + for (int i = 0; i < len; i ++) { + byteArray[i] = (byte) i; + } + this.underlying = new NettyManagedBuffer(Unpooled.wrappedBuffer(byteArray)); + } + + + @Override + public long size() { + return underlying.size(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return underlying.nioByteBuffer(); + } + + @Override + public InputStream createInputStream() throws IOException { + return underlying.createInputStream(); + } + + @Override + public ManagedBuffer retain() { + underlying.retain(); + return this; + } + + @Override + public ManagedBuffer release() { + underlying.release(); + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return underlying.convertToNetty(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ManagedBuffer) { + try { + ByteBuffer nioBuf = ((ManagedBuffer) other).nioByteBuffer(); + if (nioBuf.remaining() != len) { + return false; + } else { + for (int i = 0; i < len; i ++) { + if (nioBuf.get() != i) { + return false; + } + } + return true; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return false; + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TestUtils.java b/network/common/src/test/java/org/apache/spark/network/TestUtils.java new file mode 100644 index 0000000000000..56a2b805f154c --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/TestUtils.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.net.InetAddress; + +public class TestUtils { + public static String getLocalHost() { + try { + return InetAddress.getLocalHost().getHostAddress(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java new file mode 100644 index 0000000000000..3ef964616f0c5 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.util.concurrent.TimeoutException; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.DefaultStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.TransportConf; + +public class TransportClientFactorySuite { + private TransportConf conf; + private TransportContext context; + private TransportServer server1; + private TransportServer server2; + + @Before + public void setUp() { + conf = new TransportConf(new SystemPropertyConfigProvider()); + StreamManager streamManager = new DefaultStreamManager(); + RpcHandler rpcHandler = new NoOpRpcHandler(); + context = new TransportContext(conf, streamManager, rpcHandler); + server1 = context.createServer(); + server2 = context.createServer(); + } + + @After + public void tearDown() { + JavaUtils.closeQuietly(server1); + JavaUtils.closeQuietly(server2); + } + + @Test + public void createAndReuseBlockClients() throws TimeoutException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c3 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + assertTrue(c1.isActive()); + assertTrue(c3.isActive()); + assertTrue(c1 == c2); + assertTrue(c1 != c3); + factory.close(); + } + + @Test + public void neverReturnInactiveClients() throws Exception { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + c1.close(); + + long start = System.currentTimeMillis(); + while (c1.isActive() && (System.currentTimeMillis() - start) < 3000) { + Thread.sleep(10); + } + assertFalse(c1.isActive()); + + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertFalse(c1 == c2); + assertTrue(c2.isActive()); + factory.close(); + } + + @Test + public void closeBlockClientsWithFactory() throws TimeoutException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + assertTrue(c1.isActive()); + assertTrue(c2.isActive()); + factory.close(); + assertFalse(c1.isActive()); + assertFalse(c2.isActive()); + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java b/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java new file mode 100644 index 0000000000000..17a03ebe88a93 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import io.netty.channel.local.LocalChannel; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamChunkId; + +public class TransportResponseHandlerSuite { + @Test + public void handleSuccessfulFetch() { + StreamChunkId streamChunkId = new StreamChunkId(1, 0); + + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(streamChunkId, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchSuccess(streamChunkId, new TestManagedBuffer(123))); + verify(callback, times(1)).onSuccess(eq(0), (ManagedBuffer) any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedFetch() { + StreamChunkId streamChunkId = new StreamChunkId(1, 0); + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(streamChunkId, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchFailure(streamChunkId, "some error msg")); + verify(callback, times(1)).onFailure(eq(0), (Throwable) any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void clearAllOutstandingRequests() { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(new StreamChunkId(1, 0), callback); + handler.addFetchRequest(new StreamChunkId(1, 1), callback); + handler.addFetchRequest(new StreamChunkId(1, 2), callback); + assertEquals(3, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchSuccess(new StreamChunkId(1, 0), new TestManagedBuffer(12))); + handler.exceptionCaught(new Exception("duh duh duhhhh")); + + // should fail both b2 and b3 + verify(callback, times(1)).onSuccess(eq(0), (ManagedBuffer) any()); + verify(callback, times(1)).onFailure(eq(1), (Throwable) any()); + verify(callback, times(1)).onFailure(eq(2), (Throwable) any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleSuccessfulRPC() { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.addRpcRequest(12345, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcResponse(54321, new byte[7])); // should be ignored + assertEquals(1, handler.numOutstandingRequests()); + + byte[] arr = new byte[10]; + handler.handle(new RpcResponse(12345, arr)); + verify(callback, times(1)).onSuccess(eq(arr)); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedRPC() { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.addRpcRequest(12345, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(54321, "uh-oh!")); // should be ignored + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(12345, "oh no")); + verify(callback, times(1)).onFailure((Throwable) any()); + assertEquals(0, handler.numOutstandingRequests()); + } +} diff --git a/pom.xml b/pom.xml index abcb97108c5d9..e4c92470fc03e 100644 --- a/pom.xml +++ b/pom.xml @@ -91,6 +91,7 @@ graphx mllib tools + network/common streaming sql/catalyst sql/core diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 95152b58e287e..adbdc5d1da3c1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -51,6 +51,11 @@ object MimaExcludes { // MapStatus should be private[spark] ProblemFilters.exclude[IncompatibleTemplateDefProblem]( "org.apache.spark.scheduler.MapStatus"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.network.netty.PathResolver"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.network.netty.client.BlockClientListener"), + // TaskContext was promoted to Abstract class ProblemFilters.exclude[AbstractClassProblem]( "org.apache.spark.TaskContext"), From 353546766384b1e80fc8cc75c532d8d1821012b4 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 29 Oct 2014 12:10:58 -0700 Subject: [PATCH 077/115] [SPARK-4003] [SQL] add 3 types for java SQL context In JavaSqlContext, we need to let java program use big decimal, timestamp, date types. Author: Daoyuan Wang Closes #2850 from adrian-wang/javacontext and squashes the following commits: 4c4292c [Daoyuan Wang] change underlying type of JavaSchemaRDD as scala bb0508f [Daoyuan Wang] add test cases 3c58b0d [Daoyuan Wang] add 3 types for java SQL context --- .../spark/sql/api/java/JavaSQLContext.scala | 11 ++++- .../sql/types/util/DataTypeConversions.scala | 12 ++++++ .../spark/sql/api/java/JavaSQLSuite.scala | 41 +++++++++++++++++-- 3 files changed, 59 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index f8171c3be3207..082ae03eef03f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.types.util.DataTypeConversions import org.apache.spark.sql.{SQLContext, StructType => SStructType} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.parquet.ParquetRelation @@ -97,7 +98,9 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod) iter.map { row => - new GenericRow(extractors.map(e => e.invoke(row)).toArray[Any]): ScalaRow + new GenericRow( + extractors.map(e => DataTypeConversions.convertJavaToCatalyst(e.invoke(row))).toArray[Any] + ): ScalaRow } } new JavaSchemaRDD(sqlContext, LogicalRDD(schema, rowRdd)(sqlContext)) @@ -226,6 +229,12 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { (org.apache.spark.sql.FloatType, true) case c: Class[_] if c == classOf[java.lang.Boolean] => (org.apache.spark.sql.BooleanType, true) + case c: Class[_] if c == classOf[java.math.BigDecimal] => + (org.apache.spark.sql.DecimalType, true) + case c: Class[_] if c == classOf[java.sql.Date] => + (org.apache.spark.sql.DateType, true) + case c: Class[_] if c == classOf[java.sql.Timestamp] => + (org.apache.spark.sql.TimestampType, true) } AttributeReference(property.getName, dataType, nullable)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index e44cb08309523..609f7db562a31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -110,4 +110,16 @@ protected[sql] object DataTypeConversions { case structType: org.apache.spark.sql.api.java.StructType => StructType(structType.getFields.map(asScalaStructField)) } + + /** Converts Java objects to catalyst rows / types */ + def convertJavaToCatalyst(a: Any): Any = a match { + case d: java.math.BigDecimal => BigDecimal(d) + case other => other + } + + /** Converts Java objects to catalyst rows / types */ + def convertCatalystToJava(a: Any): Any = a match { + case d: scala.math.BigDecimal => d.underlying() + case other => other + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index 203ff847e94cc..d83f3e23a9468 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -45,6 +45,9 @@ class AllTypesBean extends Serializable { @BeanProperty var shortField: java.lang.Short = _ @BeanProperty var byteField: java.lang.Byte = _ @BeanProperty var booleanField: java.lang.Boolean = _ + @BeanProperty var dateField: java.sql.Date = _ + @BeanProperty var timestampField: java.sql.Timestamp = _ + @BeanProperty var bigDecimalField: java.math.BigDecimal = _ } class JavaSQLSuite extends FunSuite { @@ -73,6 +76,9 @@ class JavaSQLSuite extends FunSuite { bean.setShortField(0.toShort) bean.setByteField(0.toByte) bean.setBooleanField(false) + bean.setDateField(java.sql.Date.valueOf("2014-10-10")) + bean.setTimestampField(java.sql.Timestamp.valueOf("2014-10-10 00:00:00.0")) + bean.setBigDecimalField(new java.math.BigDecimal(0)) val rdd = javaCtx.parallelize(bean :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) @@ -82,10 +88,34 @@ class JavaSQLSuite extends FunSuite { javaSqlCtx.sql( """ |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField, - | booleanField + | booleanField, dateField, timestampField, bigDecimalField |FROM allTypes """.stripMargin).collect.head.row === - Seq("", 0, 0L, 0F, 0.0, 0.toShort, 0.toByte, false)) + Seq("", 0, 0L, 0F, 0.0, 0.toShort, 0.toByte, false, java.sql.Date.valueOf("2014-10-10"), + java.sql.Timestamp.valueOf("2014-10-10 00:00:00.0"), scala.math.BigDecimal(0))) + } + + test("decimal types in JavaBeans") { + val bean = new AllTypesBean + bean.setStringField("") + bean.setIntField(0) + bean.setLongField(0) + bean.setFloatField(0.0F) + bean.setDoubleField(0.0) + bean.setShortField(0.toShort) + bean.setByteField(0.toByte) + bean.setBooleanField(false) + bean.setDateField(java.sql.Date.valueOf("2014-10-10")) + bean.setTimestampField(java.sql.Timestamp.valueOf("2014-10-10 00:00:00.0")) + bean.setBigDecimalField(new java.math.BigDecimal(0)) + + val rdd = javaCtx.parallelize(bean :: Nil) + val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) + schemaRDD.registerTempTable("decimalTypes") + + assert(javaSqlCtx.sql( + "select bigDecimalField + bigDecimalField from decimalTypes" + ).collect.head.row === Seq(scala.math.BigDecimal(0))) } test("all types null in JavaBeans") { @@ -98,6 +128,9 @@ class JavaSQLSuite extends FunSuite { bean.setShortField(null) bean.setByteField(null) bean.setBooleanField(null) + bean.setDateField(null) + bean.setTimestampField(null) + bean.setBigDecimalField(null) val rdd = javaCtx.parallelize(bean :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) @@ -107,10 +140,10 @@ class JavaSQLSuite extends FunSuite { javaSqlCtx.sql( """ |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField, - | booleanField + | booleanField, dateField, timestampField, bigDecimalField |FROM allTypes """.stripMargin).collect.head.row === - Seq.fill(8)(null)) + Seq.fill(11)(null)) } test("loads JSON datasets") { From 1df05a40ebf3493b0aff46d18c0f30d2d5256c7b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 29 Oct 2014 14:01:00 -0700 Subject: [PATCH 078/115] [SPARK-3822] Executor scaling mechanism for Yarn This is part of a broader effort to enable dynamic scaling of executors ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This is intended to work alongside SPARK-3795 (#2746), SPARK-3796 and SPARK-3797, but is functionally independently of these other issues. The logic is built on top of PraveenSeluka's changes at #2798. This is different from the changes there in a few major ways: (1) the mechanism is implemented within the existing scheduler backend framework rather than in new `Actor` classes. This also introduces a parent abstract class `YarnSchedulerBackend` to encapsulate common logic to communicate with the Yarn `ApplicationMaster`. (2) The interface of requesting executors exposed to the `SparkContext` is the same, but the communication between the scheduler backend and the AM uses total number executors desired instead of an incremental number. This is discussed in #2746 and explained in the comments in the code. I have tested this significantly on a stable Yarn cluster. ------------ A remaining task for this issue is to tone down the error messages emitted when an executor is removed. Currently, `SparkContext` and its components react as if the executor has failed, resulting in many scary error messages and eventual timeouts. While it's not strictly necessary to fix this as of the first-cut implementation of this mechanism, it would be good to add logic to distinguish this case. I prefer to address this in a separate PR. I have filed a separate JIRA for this task at SPARK-4134. Author: Andrew Or Author: Andrew Or Closes #2840 from andrewor14/yarn-scaling-mechanism and squashes the following commits: 485863e [Andrew Or] Minor log message changes 4920be8 [Andrew Or] Clarify that public API is only for Yarn mode for now 1c57804 [Andrew Or] Reword a few comments + other review comments 6321140 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism 02836c0 [Andrew Or] Limit scope of synchronization 4e2ed7f [Andrew Or] Fix bug: keep track of removed executors properly 73ade46 [Andrew Or] Wording changes (minor) 2a7a6da [Andrew Or] Add `sc.killExecutor` as a shorthand (minor) 665f229 [Andrew Or] Mima excludes 79aa2df [Andrew Or] Simplify the request interface by asking for a total 04f625b [Andrew Or] Fix race condition that causes over-allocation of executors f4783f8 [Andrew Or] Change the semantics of requesting executors 005a124 [Andrew Or] Fix tests 4628b16 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism db4a679 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism 572f5c5 [Andrew Or] Unused import (minor) f30261c [Andrew Or] Kill multiple executors rather than one at a time de260d9 [Andrew Or] Simplify by skipping useless null check 9c52542 [Andrew Or] Simplify by skipping the TaskSchedulerImpl 97dd1a8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism d987b3e [Andrew Or] Move addWebUIFilters to Yarn scheduler backend 7b76d0a [Andrew Or] Expose mechanism in SparkContext as developer API 47466cd [Andrew Or] Refactor common Yarn scheduler backend logic c4dfaac [Andrew Or] Avoid thrashing when removing executors 53e8145 [Andrew Or] Start yarn actor early to listen for AM registration message bbee669 [Andrew Or] Add mechanism in yarn client mode --- .../scala/org/apache/spark/SparkContext.scala | 64 ++++++-- .../spark/scheduler/TaskSchedulerImpl.scala | 1 - .../cluster/CoarseGrainedClusterMessage.scala | 14 +- .../CoarseGrainedSchedulerBackend.scala | 105 ++++++++++--- .../cluster/YarnSchedulerBackend.scala | 142 ++++++++++++++++++ .../org/apache/spark/util/AkkaUtils.scala | 17 ++- .../SparkContextSchedulerCreationSuite.scala | 7 +- project/MimaExcludes.scala | 4 + .../spark/deploy/yarn/ApplicationMaster.scala | 34 ++++- .../spark/deploy/yarn/YarnAllocator.scala | 51 ++++++- .../cluster/YarnClientSchedulerBackend.scala | 19 +-- .../cluster/YarnClusterSchedulerBackend.scala | 12 +- 12 files changed, 391 insertions(+), 79 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e8fdfff04390d..40ea369f9ef93 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -294,7 +294,8 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler - private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) + private[spark] var (schedulerBackend, taskScheduler) = + SparkContext.createTaskScheduler(this, master) private val heartbeatReceiver = env.actorSystem.actorOf( Props(new HeartbeatReceiver(taskScheduler)), "HeartbeatReceiver") @volatile private[spark] var dagScheduler: DAGScheduler = _ @@ -856,6 +857,40 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { listenerBus.addListener(listener) } + /** + * :: DeveloperApi :: + * Request an additional number of executors from the cluster manager. + * This is currently only supported in Yarn mode. + */ + @DeveloperApi + def requestExecutors(numAdditionalExecutors: Int): Unit = { + schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => b.requestExecutors(numAdditionalExecutors) + case _ => logWarning("Requesting executors is only supported in coarse-grained mode") + } + } + + /** + * :: DeveloperApi :: + * Request that the cluster manager kill the specified executors. + * This is currently only supported in Yarn mode. + */ + @DeveloperApi + def killExecutors(executorIds: Seq[String]): Unit = { + schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => b.killExecutors(executorIds) + case _ => logWarning("Killing executors is only supported in coarse-grained mode") + } + } + + /** + * :: DeveloperApi :: + * Request that cluster manager the kill the specified executor. + * This is currently only supported in Yarn mode. + */ + @DeveloperApi + def killExecutor(executorId: String): Unit = killExecutors(Seq(executorId)) + /** The version of Spark on which this application is running. */ def version = SPARK_VERSION @@ -1438,8 +1473,13 @@ object SparkContext extends Logging { res } - /** Creates a task scheduler based on a given master URL. Extracted for testing. */ - private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { + /** + * Create a task scheduler based on a given master URL. + * Return a 2-tuple of the scheduler backend and the task scheduler. + */ + private def createTaskScheduler( + sc: SparkContext, + master: String): (SchedulerBackend, TaskScheduler) = { // Regular expression used for local[N] and local[*] master formats val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1461,7 +1501,7 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalBackend(scheduler, 1) scheduler.initialize(backend) - scheduler + (backend, scheduler) case LOCAL_N_REGEX(threads) => def localCpuCount = Runtime.getRuntime.availableProcessors() @@ -1470,7 +1510,7 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) - scheduler + (backend, scheduler) case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => def localCpuCount = Runtime.getRuntime.availableProcessors() @@ -1480,14 +1520,14 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) - scheduler + (backend, scheduler) case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) - scheduler + (backend, scheduler) case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. @@ -1507,7 +1547,7 @@ object SparkContext extends Logging { backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() } - scheduler + (backend, scheduler) case "yarn-standalone" | "yarn-cluster" => if (master == "yarn-standalone") { @@ -1536,7 +1576,7 @@ object SparkContext extends Logging { } } scheduler.initialize(backend) - scheduler + (backend, scheduler) case "yarn-client" => val scheduler = try { @@ -1563,7 +1603,7 @@ object SparkContext extends Logging { } scheduler.initialize(backend) - scheduler + (backend, scheduler) case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() @@ -1576,13 +1616,13 @@ object SparkContext extends Logging { new MesosSchedulerBackend(scheduler, sc, url) } scheduler.initialize(backend) - scheduler + (backend, scheduler) case SIMR_REGEX(simrUrl) => val scheduler = new TaskSchedulerImpl(sc) val backend = new SimrSchedulerBackend(scheduler, sc, simrUrl) scheduler.initialize(backend) - scheduler + (backend, scheduler) case _ => throw new SparkException("Could not parse Master URL: '" + master + "'") diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2b39c7fc872da..cd3c015321e85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -34,7 +34,6 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.util.Utils import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId -import akka.actor.Props /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index fb8160abc59db..1da6fe976da5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -66,7 +66,19 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage - case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase :String) + // Exchanged between the driver and the AM in Yarn client mode + case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase: String) extends CoarseGrainedClusterMessage + // Messages exchanged between the driver and the cluster manager for executor allocation + // In Yarn mode, these are exchanged between the driver and the AM + + case object RegisterClusterManager extends CoarseGrainedClusterMessage + + // Request executors by specifying the new total number of executors desired + // This includes executors already pending or running + case class RequestExecutors(requestedTotal: Int) extends CoarseGrainedClusterMessage + + case class KillExecutors(executorIds: Seq[String]) extends CoarseGrainedClusterMessage + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 59aed6b72fe42..7a6ee56f81689 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -31,7 +31,6 @@ import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState} import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} -import org.apache.spark.ui.JettyUtils /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -42,7 +41,7 @@ import org.apache.spark.ui.JettyUtils * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: ActorSystem) +class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem) extends SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed @@ -61,10 +60,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) val createTime = System.currentTimeMillis() + private val executorDataMap = new HashMap[String, ExecutorData] + + // Number of executors requested from the cluster manager that have not registered yet + private var numPendingExecutors = 0 + + // Executors we have requested the cluster manager to kill that have not died yet + private val executorsPendingToRemove = new HashSet[String] + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { override protected def log = CoarseGrainedSchedulerBackend.this.log private val addressToExecutorId = new HashMap[Address, String] - private val executorDataMap = new HashMap[String, ExecutorData] override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() @@ -84,12 +90,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } else { logInfo("Registered executor: " + sender + " with ID " + executorId) sender ! RegisteredExecutor - executorDataMap.put(executorId, new ExecutorData(sender, sender.path.address, - Utils.parseHostPort(hostPort)._1, cores, cores)) addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) + val (host, _) = Utils.parseHostPort(hostPort) + val data = new ExecutorData(sender, sender.path.address, host, cores, cores) + // This must be synchronized because variables mutated + // in this block are read when requesting executors + CoarseGrainedSchedulerBackend.this.synchronized { + executorDataMap.put(executorId, data) + if (numPendingExecutors > 0) { + numPendingExecutors -= 1 + logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") + } + } makeOffers() } @@ -128,10 +143,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A removeExecutor(executorId, reason) sender ! true - case AddWebUIFilter(filterName, filterParams, proxyBase) => - addWebUIFilter(filterName, filterParams, proxyBase) - sender ! true - case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) @@ -183,13 +194,18 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } // Remove a disconnected slave from the cluster - def removeExecutor(executorId: String, reason: String) { + def removeExecutor(executorId: String, reason: String): Unit = { executorDataMap.get(executorId) match { case Some(executorInfo) => - executorDataMap -= executorId + // This must be synchronized because variables mutated + // in this block are read when requesting executors + CoarseGrainedSchedulerBackend.this.synchronized { + executorDataMap -= executorId + executorsPendingToRemove -= executorId + } totalCoreCount.addAndGet(-executorInfo.totalCores) scheduler.executorLost(executorId, SlaveLost(reason)) - case None => logError(s"Asked to remove non existant executor $executorId") + case None => logError(s"Asked to remove non-existent executor $executorId") } } } @@ -274,21 +290,62 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A false } - // Add filters to the SparkUI - def addWebUIFilter(filterName: String, filterParams: Map[String, String], proxyBase: String) { - if (proxyBase != null && proxyBase.nonEmpty) { - System.setProperty("spark.ui.proxyBase", proxyBase) - } + /** + * Return the number of executors currently registered with this backend. + */ + def numExistingExecutors: Int = executorDataMap.size + + /** + * Request an additional number of executors from the cluster manager. + * Return whether the request is acknowledged. + */ + final def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { + logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") + logDebug(s"Number of pending executors is now $numPendingExecutors") + numPendingExecutors += numAdditionalExecutors + // Account for executors pending to be added or removed + val newTotal = numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size + doRequestTotalExecutors(newTotal) + } - val hasFilter = (filterName != null && filterName.nonEmpty && - filterParams != null && filterParams.nonEmpty) - if (hasFilter) { - logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") - conf.set("spark.ui.filters", filterName) - filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } - scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } + /** + * Request executors from the cluster manager by specifying the total number desired, + * including existing pending and running executors. + * + * The semantics here guarantee that we do not over-allocate executors for this application, + * since a later request overrides the value of any prior request. The alternative interface + * of requesting a delta of executors risks double counting new executors when there are + * insufficient resources to satisfy the first request. We make the assumption here that the + * cluster manager will eventually fulfill all requests when resources free up. + * + * Return whether the request is acknowledged. + */ + protected def doRequestTotalExecutors(requestedTotal: Int): Boolean = false + + /** + * Request that the cluster manager kill the specified executors. + * Return whether the kill request is acknowledged. + */ + final def killExecutors(executorIds: Seq[String]): Boolean = { + logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") + val filteredExecutorIds = new ArrayBuffer[String] + executorIds.foreach { id => + if (executorDataMap.contains(id)) { + filteredExecutorIds += id + } else { + logWarning(s"Executor to kill $id does not exist!") + } } + executorsPendingToRemove ++= filteredExecutorIds + doKillExecutors(filteredExecutorIds) } + + /** + * Kill the given list of executors through the cluster manager. + * Return whether the kill request is acknowledged. + */ + protected def doKillExecutors(executorIds: Seq[String]): Boolean = false + } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala new file mode 100644 index 0000000000000..50721b9d6cd6c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -0,0 +1,142 @@ +/* + * 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.scheduler.cluster + +import akka.actor.{Actor, ActorRef, Props} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} + +import org.apache.spark.SparkContext +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.ui.JettyUtils +import org.apache.spark.util.AkkaUtils + +/** + * Abstract Yarn scheduler backend that contains common logic + * between the client and cluster Yarn scheduler backends. + */ +private[spark] abstract class YarnSchedulerBackend( + scheduler: TaskSchedulerImpl, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { + + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + minRegisteredRatio = 0.8 + } + + protected var totalExpectedExecutors = 0 + + private val yarnSchedulerActor: ActorRef = + actorSystem.actorOf( + Props(new YarnSchedulerActor), + name = YarnSchedulerBackend.ACTOR_NAME) + + private implicit val askTimeout = AkkaUtils.askTimeout(sc.conf) + + /** + * Request executors from the ApplicationMaster by specifying the total number desired. + * This includes executors already pending or running. + */ + override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { + AkkaUtils.askWithReply[Boolean]( + RequestExecutors(requestedTotal), yarnSchedulerActor, askTimeout) + } + + /** + * Request that the ApplicationMaster kill the specified executors. + */ + override def doKillExecutors(executorIds: Seq[String]): Boolean = { + AkkaUtils.askWithReply[Boolean]( + KillExecutors(executorIds), yarnSchedulerActor, askTimeout) + } + + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio + } + + /** + * Add filters to the SparkUI. + */ + private def addWebUIFilter( + filterName: String, + filterParams: Map[String, String], + proxyBase: String): Unit = { + if (proxyBase != null && proxyBase.nonEmpty) { + System.setProperty("spark.ui.proxyBase", proxyBase) + } + + val hasFilter = + filterName != null && filterName.nonEmpty && + filterParams != null && filterParams.nonEmpty + if (hasFilter) { + logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") + conf.set("spark.ui.filters", filterName) + filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } + scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } + } + } + + /** + * An actor that communicates with the ApplicationMaster. + */ + private class YarnSchedulerActor extends Actor { + private var amActor: Option[ActorRef] = None + + override def preStart(): Unit = { + // Listen for disassociation events + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + } + + override def receive = { + case RegisterClusterManager => + logInfo(s"ApplicationMaster registered as $sender") + amActor = Some(sender) + + case r: RequestExecutors => + amActor match { + case Some(actor) => + sender ! AkkaUtils.askWithReply[Boolean](r, actor, askTimeout) + case None => + logWarning("Attempted to request executors before the AM has registered!") + sender ! false + } + + case k: KillExecutors => + amActor match { + case Some(actor) => + sender ! AkkaUtils.askWithReply[Boolean](k, actor, askTimeout) + case None => + logWarning("Attempted to kill executors before the AM has registered!") + sender ! false + } + + case AddWebUIFilter(filterName, filterParams, proxyBase) => + addWebUIFilter(filterName, filterParams, proxyBase) + sender ! true + + case d: DisassociatedEvent => + if (amActor.isDefined && sender == amActor.get) { + logWarning(s"ApplicationMaster has disassociated: $d") + } + } + } +} + +private[spark] object YarnSchedulerBackend { + val ACTOR_NAME = "YarnScheduler" +} diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index f41c8d0315cb3..79e398eb8c104 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -159,17 +159,28 @@ private[spark] object AkkaUtils extends Logging { def askWithReply[T]( message: Any, actor: ActorRef, - retryAttempts: Int, + timeout: FiniteDuration): T = { + askWithReply[T](message, actor, maxAttempts = 1, retryInterval = Int.MaxValue, timeout) + } + + /** + * Send a message to the given actor and get its result within a default timeout, or + * throw a SparkException if this fails even after the specified number of retries. + */ + def askWithReply[T]( + message: Any, + actor: ActorRef, + maxAttempts: Int, retryInterval: Int, timeout: FiniteDuration): T = { // TODO: Consider removing multiple attempts if (actor == null) { - throw new SparkException("Error sending message as driverActor is null " + + throw new SparkException("Error sending message as actor is null " + "[message = " + message + "]") } var attempts = 0 var lastException: Exception = null - while (attempts < retryAttempts) { + while (attempts < maxAttempts) { attempts += 1 try { val future = actor.ask(message)(timeout) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 495a0d48633a4..df237ba796b38 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import org.scalatest.{BeforeAndAfterEach, FunSuite, PrivateMethodTester} -import org.apache.spark.scheduler.{TaskScheduler, TaskSchedulerImpl} +import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend @@ -31,8 +31,9 @@ class SparkContextSchedulerCreationSuite // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. val sc = new SparkContext("local", "test") - val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) - val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) + val createTaskSchedulerMethod = + PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) + val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index adbdc5d1da3c1..6a0495f8fd540 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -73,6 +73,10 @@ object MimaExcludes { "org.apache.spark.api.java.JavaRDDLike.foreachAsync"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.api.java.JavaRDDLike.collectAsync") + ) ++ Seq( + // SPARK-3822 + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") ) case v if v.startsWith("1.1") => diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index e6fe0265d8811..68073798886dd 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -36,8 +36,8 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, Spar import org.apache.spark.SparkException import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter +import org.apache.spark.scheduler.cluster.YarnSchedulerBackend +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} /** @@ -385,8 +385,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, SparkEnv.driverActorSystemName, driverHost, driverPort.toString, - CoarseGrainedSchedulerBackend.ACTOR_NAME) - actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + YarnSchedulerBackend.ACTOR_NAME) + actorSystem.actorOf(Props(new AMActor(driverUrl)), name = "YarnAM") } /** Add the Yarn IP filter that is required for properly securing the UI. */ @@ -479,9 +479,10 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, userThread } - // Actor used to monitor the driver when running in client deploy mode. - private class MonitorActor(driverUrl: String) extends Actor { - + /** + * Actor that communicates with the driver in client deploy mode. + */ + private class AMActor(driverUrl: String) extends Actor { var driver: ActorSelection = _ override def preStart() = { @@ -490,6 +491,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // Send a hello message to establish the connection, after which // we can monitor Lifecycle Events. driver ! "Hello" + driver ! RegisterClusterManager context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } @@ -497,11 +499,27 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + case x: AddWebUIFilter => logInfo(s"Add WebUI Filter. $x") driver ! x - } + case RequestExecutors(requestedTotal) => + logInfo(s"Driver requested a total number of executors of $requestedTotal.") + Option(allocator) match { + case Some(a) => a.requestTotalExecutors(requestedTotal) + case None => logWarning("Container allocator is not ready to request executors yet.") + } + sender ! true + + case KillExecutors(executorIds) => + logInfo(s"Driver requested to kill executor(s) ${executorIds.mkString(", ")}.") + Option(allocator) match { + case Some(a) => executorIds.foreach(a.killExecutor) + case None => logWarning("Container allocator is not ready to kill executors yet.") + } + sender ! true + } } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index e1af8d5a74cb1..7ae8ef237ff89 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -88,7 +88,10 @@ private[yarn] abstract class YarnAllocator( private val executorIdCounter = new AtomicInteger() private val numExecutorsFailed = new AtomicInteger() - private val maxExecutors = args.numExecutors + private var maxExecutors = args.numExecutors + + // Keep track of which container is running which executor to remove the executors later + private val executorIdToContainer = new HashMap[String, Container] protected val executorMemory = args.executorMemory protected val executorCores = args.executorCores @@ -111,7 +114,48 @@ private[yarn] abstract class YarnAllocator( def getNumExecutorsFailed: Int = numExecutorsFailed.intValue - def allocateResources() = { + /** + * Request as many executors from the ResourceManager as needed to reach the desired total. + * This takes into account executors already running or pending. + */ + def requestTotalExecutors(requestedTotal: Int): Unit = synchronized { + val currentTotal = numPendingAllocate.get + numExecutorsRunning.get + if (requestedTotal > currentTotal) { + maxExecutors += (requestedTotal - currentTotal) + // We need to call `allocateResources` here to avoid the following race condition: + // If we request executors twice before `allocateResources` is called, then we will end up + // double counting the number requested because `numPendingAllocate` is not updated yet. + allocateResources() + } else { + logInfo(s"Not allocating more executors because there are already $currentTotal " + + s"(application requested $requestedTotal total)") + } + } + + /** + * Request that the ResourceManager release the container running the specified executor. + */ + def killExecutor(executorId: String): Unit = synchronized { + if (executorIdToContainer.contains(executorId)) { + val container = executorIdToContainer.remove(executorId).get + internalReleaseContainer(container) + numExecutorsRunning.decrementAndGet() + maxExecutors -= 1 + assert(maxExecutors >= 0, "Allocator killed more executors than are allocated!") + } else { + logWarning(s"Attempted to kill unknown executor $executorId!") + } + } + + /** + * Allocate missing containers based on the number of executors currently pending and running. + * + * This method prioritizes the allocated container responses from the RM based on node and + * rack locality. Additionally, it releases any extra containers allocated for this application + * but are not needed. This must be synchronized because variables read in this block are + * mutated by other methods. + */ + def allocateResources(): Unit = synchronized { val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() // this is needed by alpha, do it here since we add numPending right after this @@ -119,7 +163,7 @@ private[yarn] abstract class YarnAllocator( if (missing > 0) { val totalExecutorMemory = executorMemory + memoryOverhead numPendingAllocate.addAndGet(missing) - logInfo(s"Will allocate $missing executor containers, each with $totalExecutorMemory MB " + + logInfo(s"Will allocate $missing executor containers, each with $totalExecutorMemory MB " + s"memory including $memoryOverhead MB overhead") } else { logDebug("Empty allocation request ...") @@ -269,6 +313,7 @@ private[yarn] abstract class YarnAllocator( CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) + executorIdToContainer(executorId) = container // To be safe, remove the container from `releasedContainers`. releasedContainers.remove(containerId) diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 59b2b47aed2fe..f6f6dc52433e5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -17,27 +17,23 @@ package org.apache.spark.scheduler.cluster +import scala.collection.mutable.ArrayBuffer + import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} + import org.apache.spark.{SparkException, Logging, SparkContext} import org.apache.spark.deploy.yarn.{Client, ClientArguments} import org.apache.spark.scheduler.TaskSchedulerImpl -import scala.collection.mutable.ArrayBuffer - private[spark] class YarnClientSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends YarnSchedulerBackend(scheduler, sc) with Logging { - if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { - minRegisteredRatio = 0.8 - } - private var client: Client = null private var appId: ApplicationId = null private var stopping: Boolean = false - private var totalExpectedExecutors = 0 /** * Create a Yarn client to submit an application to the ResourceManager. @@ -151,14 +147,11 @@ private[spark] class YarnClientSchedulerBackend( logInfo("Stopped") } - override def sufficientResourcesRegistered(): Boolean = { - totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio - } - - override def applicationId(): String = + override def applicationId(): String = { Option(appId).map(_.toString).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } + } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 3a186cfeb4eeb..a96a54f66824c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -25,13 +25,7 @@ import org.apache.spark.util.IntParam private[spark] class YarnClusterSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { - - var totalExpectedExecutors = 0 - - if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { - minRegisteredRatio = 0.8 - } + extends YarnSchedulerBackend(scheduler, sc) { override def start() { super.start() @@ -44,10 +38,6 @@ private[spark] class YarnClusterSchedulerBackend( totalExpectedExecutors = sc.getConf.getInt("spark.executor.instances", totalExpectedExecutors) } - override def sufficientResourcesRegistered(): Boolean = { - totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio - } - override def applicationId(): String = // In YARN Cluster mode, spark.yarn.app.id is expect to be set // before user application is launched. From e7fd80413d531e23b6c4def0ee32e52a39da36fa Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 29 Oct 2014 14:42:50 -0700 Subject: [PATCH 079/115] [SPARK-4097] Fix the race condition of 'thread' There is a chance that `thread` is null when calling `thread.interrupt()`. ```Scala override def cancel(): Unit = this.synchronized { _cancelled = true if (thread != null) { thread.interrupt() } } ``` Should put `thread = null` into a `synchronized` block to fix the race condition. Author: zsxwing Closes #2957 from zsxwing/SPARK-4097 and squashes the following commits: edf0aee [zsxwing] Add comments to explain the lock c5cfeca [zsxwing] Fix the race condition of 'thread' --- core/src/main/scala/org/apache/spark/FutureAction.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index d5c8f9d76c476..e97a7375a267b 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -210,7 +210,11 @@ class ComplexFutureAction[T] extends FutureAction[T] { } catch { case e: Exception => p.failure(e) } finally { - thread = null + // This lock guarantees when calling `thread.interrupt()` in `cancel`, + // thread won't be set to null. + ComplexFutureAction.this.synchronized { + thread = null + } } } this From 8d59b37b02eb36f37bcefafb952519d7dca744ad Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 29 Oct 2014 17:48:59 -0700 Subject: [PATCH 080/115] [SPARK-3795] Heuristics for dynamically scaling executors This is part of a bigger effort to provide elastic scaling of executors within a Spark application ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This PR does not provide any functionality by itself; it is a skeleton that is missing a mechanism to be added later in [SPARK-3822](https://issues.apache.org/jira/browse/SPARK-3822). Comments and feedback are most welcome. For those of you reviewing this in detail, I highly recommend doing it through your favorite IDE instead of through the diff here. Author: Andrew Or Author: Andrew Or Closes #2746 from andrewor14/scaling-heuristics and squashes the following commits: 8a4fdaa [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics e045df8 [Andrew Or] Add warning message (minor) dfa31ec [Andrew Or] Fix tests c0becc4 [Andrew Or] Merging with SPARK-3822 4784f93 [Andrew Or] Reword an awkward log message 181f27f [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics c79e907 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 4672b90 [Andrew Or] It's nano time. a6a30f2 [Andrew Or] Do not allow min/max executors of 0 c60ec33 [Andrew Or] Rewrite test logic with clocks b00b680 [Andrew Or] Fix style c3caa65 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 7f9da14 [Andrew Or] Factor out logic to verify bounds on # executors (minor) f279019 [Andrew Or] Add time mocking tests for polling loop 685e347 [Andrew Or] Factor out clock in polling loop to facilitate testing 3cea7f7 [Andrew Or] Use PrivateMethodTester to keep original class private 3156d81 [Andrew Or] Update comments and exception messages 92f36f9 [Andrew Or] Address minor review comments abdea61 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 2aefd09 [Andrew Or] Correct listener behavior 9fe6e44 [Andrew Or] Rename variables and configs + update comments and log messages 149cc32 [Andrew Or] Fix style 254c958 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 5ff829b [Andrew Or] Add tests for ExecutorAllocationManager 19c6c4b [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 5896515 [Andrew Or] Move ExecutorAllocationManager out of scheduler package 9ca8945 [Andrew Or] Rewrite callbacks through the listener interface 5e336b9 [Andrew Or] Remove code from backend to avoid conflict with SPARK-3822 092d1fd [Andrew Or] Remove timeout logic for pending requests 1309fab [Andrew Or] Request executors by specifying the number pending 8bc0e9d [Andrew Or] Add logic to expire pending requests after timeouts b750ee1 [Andrew Or] Express timers in terms of expiration times + remove retry logic 7f8dd47 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 9d516cc [Andrew Or] Bug fix: Actually trigger the add timer / add retry timer 44f1832 [Andrew Or] Rename configs to include time units eaae7ef [Andrew Or] Address various review comments 6f8be6c [Andrew Or] Beef up comments on what each of the timers mean baaa403 [Andrew Or] Simplify variable names (minor) 42beec8 [Andrew Or] Reset whether the add threshold is crossed on cancellation 9bcc0bc [Andrew Or] ExecutorScalingManager -> ExecutorAllocationManager 2784398 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 5a97d9e [Andrew Or] Log retry attempts in INFO + clean up logging 2f55c9f [Andrew Or] Do not keep requesting executors even after max attempts 0acd1cb [Andrew Or] Rewrite timer logic with polling b3c7d44 [Andrew Or] Start the retry timer for adding executors at the right time 9b5f2ea [Andrew Or] Wording changes in comments and log messages c2203a5 [Andrew Or] Simplify code to access the scheduler backend e519d08 [Andrew Or] Simplify initialization code 2cc87a7 [Andrew Or] Add retry logic for removing executors d0b34a6 [Andrew Or] Add retry logic for adding executors 9cc4649 [Andrew Or] Simplifying synchronization logic 67c03c7 [Andrew Or] Correct semantics of adding executors + update comments 6c48ab0 [Andrew Or] Update synchronization comment 8901900 [Andrew Or] Simplify remove policy + change the semantics of add policy 1cc8444 [Andrew Or] Minor wording change ae5b64a [Andrew Or] Add synchronization 20ec6b9 [Andrew Or] First cut implementation of removing executors dynamically 4077ae2 [Andrew Or] Minor code re-organization 6f1fa66 [Andrew Or] First cut implementation of adding executors dynamically b2e6dcc [Andrew Or] Add skeleton interface for requesting / killing executors --- .../spark/ExecutorAllocationManager.scala | 462 ++++++++++++ .../scala/org/apache/spark/SparkContext.scala | 35 +- .../ExecutorAllocationManagerSuite.scala | 662 ++++++++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- 4 files changed, 1150 insertions(+), 11 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala create mode 100644 core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala new file mode 100644 index 0000000000000..b2cf022baf29f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -0,0 +1,462 @@ +/* + * 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 scala.collection.mutable + +import org.apache.spark.scheduler._ + +/** + * An agent that dynamically allocates and removes executors based on the workload. + * + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue + * persists for another M seconds, then more executors are added and so on. The number added + * in each round increases exponentially from the previous round until an upper bound on the + * number of executors has been reached. + * + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise, + * we may add more executors than we need just to remove them later. (2) Executors should be added + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take + * a long time to ramp up under heavy workloads. + * + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not + * been scheduled to run any tasks, then it is removed. + * + * There is no retry logic in either case because we make the assumption that the cluster manager + * will eventually fulfill all requests it receives asynchronously. + * + * The relevant Spark properties include the following: + * + * spark.dynamicAllocation.enabled - Whether this feature is enabled + * spark.dynamicAllocation.minExecutors - Lower bound on the number of executors + * spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors + * + * spark.dynamicAllocation.schedulerBacklogTimeout (M) - + * If there are backlogged tasks for this duration, add new executors + * + * spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) - + * If the backlog is sustained for this duration, add more executors + * This is used only after the initial backlog timeout is exceeded + * + * spark.dynamicAllocation.executorIdleTimeout (K) - + * If an executor has been idle for this duration, remove it + */ +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging { + import ExecutorAllocationManager._ + + private val conf = sc.conf + + // Lower and upper bounds on the number of executors. These are required. + private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1) + private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1) + verifyBounds() + + // How long there must be backlogged tasks for before an addition is triggered + private val schedulerBacklogTimeout = conf.getLong( + "spark.dynamicAllocation.schedulerBacklogTimeout", 60) + + // Same as above, but used only after `schedulerBacklogTimeout` is exceeded + private val sustainedSchedulerBacklogTimeout = conf.getLong( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) + + // How long an executor must be idle for before it is removed + private val removeThresholdSeconds = conf.getLong( + "spark.dynamicAllocation.executorIdleTimeout", 600) + + // Number of executors to add in the next round + private var numExecutorsToAdd = 1 + + // Number of executors that have been requested but have not registered yet + private var numExecutorsPending = 0 + + // Executors that have been requested to be removed but have not been killed yet + private val executorsPendingToRemove = new mutable.HashSet[String] + + // All known executors + private val executorIds = new mutable.HashSet[String] + + // A timestamp of when an addition should be triggered, or NOT_SET if it is not set + // This is set when pending tasks are added but not scheduled yet + private var addTime: Long = NOT_SET + + // A timestamp for each executor of when the executor should be removed, indexed by the ID + // This is set when an executor is no longer running a task, or when it first registers + private val removeTimes = new mutable.HashMap[String, Long] + + // Polling loop interval (ms) + private val intervalMillis: Long = 100 + + // Whether we are testing this class. This should only be used internally. + private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) + + // Clock used to schedule when executors should be added and removed + private var clock: Clock = new RealClock + + /** + * Verify that the lower and upper bounds on the number of executors are valid. + * If not, throw an appropriate exception. + */ + private def verifyBounds(): Unit = { + if (minNumExecutors < 0 || maxNumExecutors < 0) { + throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!") + } + if (minNumExecutors == 0 || maxNumExecutors == 0) { + throw new SparkException("spark.dynamicAllocation.{min/max}Executors cannot be 0!") + } + if (minNumExecutors > maxNumExecutors) { + throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + + s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") + } + } + + /** + * Use a different clock for this allocation manager. This is mainly used for testing. + */ + def setClock(newClock: Clock): Unit = { + clock = newClock + } + + /** + * Register for scheduler callbacks to decide when to add and remove executors. + */ + def start(): Unit = { + val listener = new ExecutorAllocationListener(this) + sc.addSparkListener(listener) + startPolling() + } + + /** + * Start the main polling thread that keeps track of when to add and remove executors. + */ + private def startPolling(): Unit = { + val t = new Thread { + override def run(): Unit = { + while (true) { + try { + schedule() + } catch { + case e: Exception => logError("Exception in dynamic executor allocation thread!", e) + } + Thread.sleep(intervalMillis) + } + } + } + t.setName("spark-dynamic-executor-allocation") + t.setDaemon(true) + t.start() + } + + /** + * If the add time has expired, request new executors and refresh the add time. + * If the remove time for an existing executor has expired, kill the executor. + * This is factored out into its own method for testing. + */ + private def schedule(): Unit = synchronized { + val now = clock.getTimeMillis + if (addTime != NOT_SET && now >= addTime) { + addExecutors() + logDebug(s"Starting timer to add more executors (to " + + s"expire in $sustainedSchedulerBacklogTimeout seconds)") + addTime += sustainedSchedulerBacklogTimeout * 1000 + } + + removeTimes.foreach { case (executorId, expireTime) => + if (now >= expireTime) { + removeExecutor(executorId) + removeTimes.remove(executorId) + } + } + } + + /** + * Request a number of executors from the cluster manager. + * If the cap on the number of executors is reached, give up and reset the + * number of executors to add next round instead of continuing to double it. + * Return the number actually requested. + */ + private def addExecutors(): Int = synchronized { + // Do not request more executors if we have already reached the upper bound + val numExistingExecutors = executorIds.size + numExecutorsPending + if (numExistingExecutors >= maxNumExecutors) { + logDebug(s"Not adding executors because there are already ${executorIds.size} " + + s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)") + numExecutorsToAdd = 1 + return 0 + } + + // Request executors with respect to the upper bound + val actualNumExecutorsToAdd = + if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) { + numExecutorsToAdd + } else { + maxNumExecutors - numExistingExecutors + } + val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd + val addRequestAcknowledged = testing || sc.requestExecutors(actualNumExecutorsToAdd) + if (addRequestAcknowledged) { + logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " + + s"tasks are backlogged (new desired total will be $newTotalExecutors)") + numExecutorsToAdd = + if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1 + numExecutorsPending += actualNumExecutorsToAdd + actualNumExecutorsToAdd + } else { + logWarning(s"Unable to reach the cluster manager " + + s"to request $actualNumExecutorsToAdd executors!") + 0 + } + } + + /** + * Request the cluster manager to remove the given executor. + * Return whether the request is received. + */ + private def removeExecutor(executorId: String): Boolean = synchronized { + // Do not kill the executor if we are not aware of it (should never happen) + if (!executorIds.contains(executorId)) { + logWarning(s"Attempted to remove unknown executor $executorId!") + return false + } + + // Do not kill the executor again if it is already pending to be killed (should never happen) + if (executorsPendingToRemove.contains(executorId)) { + logWarning(s"Attempted to remove executor $executorId " + + s"when it is already pending to be removed!") + return false + } + + // Do not kill the executor if we have already reached the lower bound + val numExistingExecutors = executorIds.size - executorsPendingToRemove.size + if (numExistingExecutors - 1 < minNumExecutors) { + logInfo(s"Not removing idle executor $executorId because there are only " + + s"$numExistingExecutors executor(s) left (limit $minNumExecutors)") + return false + } + + // Send a request to the backend to kill this executor + val removeRequestAcknowledged = testing || sc.killExecutor(executorId) + if (removeRequestAcknowledged) { + logInfo(s"Removing executor $executorId because it has been idle for " + + s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})") + executorsPendingToRemove.add(executorId) + true + } else { + logWarning(s"Unable to reach the cluster manager to kill executor $executorId!") + false + } + } + + /** + * Callback invoked when the specified executor has been added. + */ + private def onExecutorAdded(executorId: String): Unit = synchronized { + if (!executorIds.contains(executorId)) { + executorIds.add(executorId) + executorIds.foreach(onExecutorIdle) + logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})") + if (numExecutorsPending > 0) { + numExecutorsPending -= 1 + logDebug(s"Decremented number of pending executors ($numExecutorsPending left)") + } + } else { + logWarning(s"Duplicate executor $executorId has registered") + } + } + + /** + * Callback invoked when the specified executor has been removed. + */ + private def onExecutorRemoved(executorId: String): Unit = synchronized { + if (executorIds.contains(executorId)) { + executorIds.remove(executorId) + removeTimes.remove(executorId) + logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})") + if (executorsPendingToRemove.contains(executorId)) { + executorsPendingToRemove.remove(executorId) + logDebug(s"Executor $executorId is no longer pending to " + + s"be removed (${executorsPendingToRemove.size} left)") + } + } else { + logWarning(s"Unknown executor $executorId has been removed!") + } + } + + /** + * Callback invoked when the scheduler receives new pending tasks. + * This sets a time in the future that decides when executors should be added + * if it is not already set. + */ + private def onSchedulerBacklogged(): Unit = synchronized { + if (addTime == NOT_SET) { + logDebug(s"Starting timer to add executors because pending tasks " + + s"are building up (to expire in $schedulerBacklogTimeout seconds)") + addTime = clock.getTimeMillis + schedulerBacklogTimeout * 1000 + } + } + + /** + * Callback invoked when the scheduler queue is drained. + * This resets all variables used for adding executors. + */ + private def onSchedulerQueueEmpty(): Unit = synchronized { + logDebug(s"Clearing timer to add executors because there are no more pending tasks") + addTime = NOT_SET + numExecutorsToAdd = 1 + } + + /** + * Callback invoked when the specified executor is no longer running any tasks. + * This sets a time in the future that decides when this executor should be removed if + * the executor is not already marked as idle. + */ + private def onExecutorIdle(executorId: String): Unit = synchronized { + if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { + logDebug(s"Starting idle timer for $executorId because there are no more tasks " + + s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)") + removeTimes(executorId) = clock.getTimeMillis + removeThresholdSeconds * 1000 + } + } + + /** + * Callback invoked when the specified executor is now running a task. + * This resets all variables used for removing this executor. + */ + private def onExecutorBusy(executorId: String): Unit = synchronized { + logDebug(s"Clearing idle timer for $executorId because it is now running a task") + removeTimes.remove(executorId) + } + + /** + * A listener that notifies the given allocation manager of when to add and remove executors. + * + * This class is intentionally conservative in its assumptions about the relative ordering + * and consistency of events returned by the listener. For simplicity, it does not account + * for speculated tasks. + */ + private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager) + extends SparkListener { + + private val stageIdToNumTasks = new mutable.HashMap[Int, Int] + private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] + private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]] + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + synchronized { + val stageId = stageSubmitted.stageInfo.stageId + val numTasks = stageSubmitted.stageInfo.numTasks + stageIdToNumTasks(stageId) = numTasks + allocationManager.onSchedulerBacklogged() + } + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + synchronized { + val stageId = stageCompleted.stageInfo.stageId + stageIdToNumTasks -= stageId + stageIdToTaskIndices -= stageId + + // If this is the last stage with pending tasks, mark the scheduler queue as empty + // This is needed in case the stage is aborted for any reason + if (stageIdToNumTasks.isEmpty) { + allocationManager.onSchedulerQueueEmpty() + } + } + } + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { + val stageId = taskStart.stageId + val taskId = taskStart.taskInfo.taskId + val taskIndex = taskStart.taskInfo.index + val executorId = taskStart.taskInfo.executorId + + // If this is the last pending task, mark the scheduler queue as empty + stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex + val numTasksScheduled = stageIdToTaskIndices(stageId).size + val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1) + if (numTasksScheduled == numTasksTotal) { + // No more pending tasks for this stage + stageIdToNumTasks -= stageId + if (stageIdToNumTasks.isEmpty) { + allocationManager.onSchedulerQueueEmpty() + } + } + + // Mark the executor on which this task is scheduled as busy + executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId + allocationManager.onExecutorBusy(executorId) + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + val executorId = taskEnd.taskInfo.executorId + val taskId = taskEnd.taskInfo.taskId + + // If the executor is no longer running scheduled any tasks, mark it as idle + if (executorIdToTaskIds.contains(executorId)) { + executorIdToTaskIds(executorId) -= taskId + if (executorIdToTaskIds(executorId).isEmpty) { + executorIdToTaskIds -= executorId + allocationManager.onExecutorIdle(executorId) + } + } + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { + val executorId = blockManagerAdded.blockManagerId.executorId + if (executorId != "") { + allocationManager.onExecutorAdded(executorId) + } + } + + override def onBlockManagerRemoved( + blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { + allocationManager.onExecutorRemoved(blockManagerRemoved.blockManagerId.executorId) + } + } + +} + +private object ExecutorAllocationManager { + val NOT_SET = Long.MaxValue +} + +/** + * An abstract clock for measuring elapsed time. + */ +private trait Clock { + def getTimeMillis: Long +} + +/** + * A clock backed by a monotonically increasing time source. + * The time returned by this clock does not correspond to any notion of wall-clock time. + */ +private class RealClock extends Clock { + override def getTimeMillis: Long = System.nanoTime / (1000 * 1000) +} + +/** + * A clock that allows the caller to customize the time. + * This is used mainly for testing. + */ +private class TestClock(startTimeMillis: Long) extends Clock { + private var time: Long = startTimeMillis + override def getTimeMillis: Long = time + def tick(ms: Long): Unit = { time += ms } +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 40ea369f9ef93..73668e83bbb1d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -330,6 +330,15 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { } else None } + // Optionally scale number of executors dynamically based on workload. Exposed for testing. + private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = + if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { + Some(new ExecutorAllocationManager(this)) + } else { + None + } + executorAllocationManager.foreach(_.start()) + // At this point, all relevant SparkListeners have been registered, so begin releasing events listenerBus.start() @@ -860,36 +869,42 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { /** * :: DeveloperApi :: * Request an additional number of executors from the cluster manager. - * This is currently only supported in Yarn mode. + * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def requestExecutors(numAdditionalExecutors: Int): Unit = { + def requestExecutors(numAdditionalExecutors: Int): Boolean = { schedulerBackend match { - case b: CoarseGrainedSchedulerBackend => b.requestExecutors(numAdditionalExecutors) - case _ => logWarning("Requesting executors is only supported in coarse-grained mode") + case b: CoarseGrainedSchedulerBackend => + b.requestExecutors(numAdditionalExecutors) + case _ => + logWarning("Requesting executors is only supported in coarse-grained mode") + false } } /** * :: DeveloperApi :: * Request that the cluster manager kill the specified executors. - * This is currently only supported in Yarn mode. + * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def killExecutors(executorIds: Seq[String]): Unit = { + def killExecutors(executorIds: Seq[String]): Boolean = { schedulerBackend match { - case b: CoarseGrainedSchedulerBackend => b.killExecutors(executorIds) - case _ => logWarning("Killing executors is only supported in coarse-grained mode") + case b: CoarseGrainedSchedulerBackend => + b.killExecutors(executorIds) + case _ => + logWarning("Killing executors is only supported in coarse-grained mode") + false } } /** * :: DeveloperApi :: * Request that cluster manager the kill the specified executor. - * This is currently only supported in Yarn mode. + * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def killExecutor(executorId: String): Unit = killExecutors(Seq(executorId)) + def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId)) /** The version of Spark on which this application is running. */ def version = SPARK_VERSION diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala new file mode 100644 index 0000000000000..f0aa914cfedb6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -0,0 +1,662 @@ +/* + * 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 org.scalatest.{FunSuite, PrivateMethodTester} +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler._ +import org.apache.spark.storage.BlockManagerId + +/** + * Test add and remove behavior of ExecutorAllocationManager. + */ +class ExecutorAllocationManagerSuite extends FunSuite { + import ExecutorAllocationManager._ + import ExecutorAllocationManagerSuite._ + + test("verify min/max executors") { + // No min or max + val conf = new SparkConf() + .setMaster("local") + .setAppName("test-executor-allocation-manager") + .set("spark.dynamicAllocation.enabled", "true") + intercept[SparkException] { new SparkContext(conf) } + + // Only min + val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1") + intercept[SparkException] { new SparkContext(conf1) } + + // Only max + val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2") + intercept[SparkException] { new SparkContext(conf2) } + + // Both min and max, but min > max + intercept[SparkException] { createSparkContext(2, 1) } + + // Both min and max, and min == max + val sc1 = createSparkContext(1, 1) + assert(sc1.executorAllocationManager.isDefined) + sc1.stop() + + // Both min and max, and min < max + val sc2 = createSparkContext(1, 2) + assert(sc2.executorAllocationManager.isDefined) + sc2.stop() + } + + test("starting state") { + val sc = createSparkContext() + val manager = sc.executorAllocationManager.get + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + assert(executorIds(manager).isEmpty) + assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) + assert(removeTimes(manager).isEmpty) + sc.stop() + } + + test("add executors") { + val sc = createSparkContext(1, 10) + val manager = sc.executorAllocationManager.get + + // Keep adding until the limit is reached + assert(numExecutorsPending(manager) === 0) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 1) + assert(numExecutorsPending(manager) === 1) + assert(numExecutorsToAdd(manager) === 2) + assert(addExecutors(manager) === 2) + assert(numExecutorsPending(manager) === 3) + assert(numExecutorsToAdd(manager) === 4) + assert(addExecutors(manager) === 4) + assert(numExecutorsPending(manager) === 7) + assert(numExecutorsToAdd(manager) === 8) + assert(addExecutors(manager) === 3) // reached the limit of 10 + assert(numExecutorsPending(manager) === 10) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 0) + assert(numExecutorsPending(manager) === 10) + assert(numExecutorsToAdd(manager) === 1) + + // Register previously requested executors + onExecutorAdded(manager, "first") + assert(numExecutorsPending(manager) === 9) + onExecutorAdded(manager, "second") + onExecutorAdded(manager, "third") + onExecutorAdded(manager, "fourth") + assert(numExecutorsPending(manager) === 6) + onExecutorAdded(manager, "first") // duplicates should not count + onExecutorAdded(manager, "second") + assert(numExecutorsPending(manager) === 6) + + // Try adding again + // This should still fail because the number pending + running is still at the limit + assert(addExecutors(manager) === 0) + assert(numExecutorsPending(manager) === 6) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 0) + assert(numExecutorsPending(manager) === 6) + assert(numExecutorsToAdd(manager) === 1) + sc.stop() + } + + test("remove executors") { + val sc = createSparkContext(5, 10) + val manager = sc.executorAllocationManager.get + (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + + // Keep removing until the limit is reached + assert(executorsPendingToRemove(manager).isEmpty) + assert(removeExecutor(manager, "1")) + assert(executorsPendingToRemove(manager).size === 1) + assert(executorsPendingToRemove(manager).contains("1")) + assert(removeExecutor(manager, "2")) + assert(removeExecutor(manager, "3")) + assert(executorsPendingToRemove(manager).size === 3) + assert(executorsPendingToRemove(manager).contains("2")) + assert(executorsPendingToRemove(manager).contains("3")) + assert(!removeExecutor(manager, "100")) // remove non-existent executors + assert(!removeExecutor(manager, "101")) + assert(executorsPendingToRemove(manager).size === 3) + assert(removeExecutor(manager, "4")) + assert(removeExecutor(manager, "5")) + assert(!removeExecutor(manager, "6")) // reached the limit of 5 + assert(executorsPendingToRemove(manager).size === 5) + assert(executorsPendingToRemove(manager).contains("4")) + assert(executorsPendingToRemove(manager).contains("5")) + assert(!executorsPendingToRemove(manager).contains("6")) + + // Kill executors previously requested to remove + onExecutorRemoved(manager, "1") + assert(executorsPendingToRemove(manager).size === 4) + assert(!executorsPendingToRemove(manager).contains("1")) + onExecutorRemoved(manager, "2") + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 2) + assert(!executorsPendingToRemove(manager).contains("2")) + assert(!executorsPendingToRemove(manager).contains("3")) + onExecutorRemoved(manager, "2") // duplicates should not count + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 2) + onExecutorRemoved(manager, "4") + onExecutorRemoved(manager, "5") + assert(executorsPendingToRemove(manager).isEmpty) + + // Try removing again + // This should still fail because the number pending + running is still at the limit + assert(!removeExecutor(manager, "7")) + assert(executorsPendingToRemove(manager).isEmpty) + assert(!removeExecutor(manager, "8")) + assert(executorsPendingToRemove(manager).isEmpty) + sc.stop() + } + + test ("interleaving add and remove") { + val sc = createSparkContext(5, 10) + val manager = sc.executorAllocationManager.get + + // Add a few executors + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 2) + assert(addExecutors(manager) === 4) + onExecutorAdded(manager, "1") + onExecutorAdded(manager, "2") + onExecutorAdded(manager, "3") + onExecutorAdded(manager, "4") + onExecutorAdded(manager, "5") + onExecutorAdded(manager, "6") + onExecutorAdded(manager, "7") + assert(executorIds(manager).size === 7) + + // Remove until limit + assert(removeExecutor(manager, "1")) + assert(removeExecutor(manager, "2")) + assert(!removeExecutor(manager, "3")) // lower limit reached + assert(!removeExecutor(manager, "4")) + onExecutorRemoved(manager, "1") + onExecutorRemoved(manager, "2") + assert(executorIds(manager).size === 5) + + // Add until limit + assert(addExecutors(manager) === 5) // upper limit reached + assert(addExecutors(manager) === 0) + assert(!removeExecutor(manager, "3")) // still at lower limit + assert(!removeExecutor(manager, "4")) + onExecutorAdded(manager, "8") + onExecutorAdded(manager, "9") + onExecutorAdded(manager, "10") + onExecutorAdded(manager, "11") + onExecutorAdded(manager, "12") + assert(executorIds(manager).size === 10) + + // Remove succeeds again, now that we are no longer at the lower limit + assert(removeExecutor(manager, "3")) + assert(removeExecutor(manager, "4")) + assert(removeExecutor(manager, "5")) + assert(removeExecutor(manager, "6")) + assert(executorIds(manager).size === 10) + assert(addExecutors(manager) === 0) // still at upper limit + onExecutorRemoved(manager, "3") + onExecutorRemoved(manager, "4") + assert(executorIds(manager).size === 8) + + // Add succeeds again, now that we are no longer at the upper limit + // Number of executors added restarts at 1 + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 1) // upper limit reached again + assert(addExecutors(manager) === 0) + assert(executorIds(manager).size === 8) + onExecutorRemoved(manager, "5") + onExecutorRemoved(manager, "6") + onExecutorAdded(manager, "13") + onExecutorAdded(manager, "14") + assert(executorIds(manager).size === 8) + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 1) // upper limit reached again + assert(addExecutors(manager) === 0) + onExecutorAdded(manager, "15") + onExecutorAdded(manager, "16") + assert(executorIds(manager).size === 10) + sc.stop() + } + + test("starting/canceling add timer") { + val sc = createSparkContext(2, 10) + val clock = new TestClock(8888L) + val manager = sc.executorAllocationManager.get + manager.setClock(clock) + + // Starting add timer is idempotent + assert(addTime(manager) === NOT_SET) + onSchedulerBacklogged(manager) + val firstAddTime = addTime(manager) + assert(firstAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) + clock.tick(100L) + onSchedulerBacklogged(manager) + assert(addTime(manager) === firstAddTime) // timer is already started + clock.tick(200L) + onSchedulerBacklogged(manager) + assert(addTime(manager) === firstAddTime) + onSchedulerQueueEmpty(manager) + + // Restart add timer + clock.tick(1000L) + assert(addTime(manager) === NOT_SET) + onSchedulerBacklogged(manager) + val secondAddTime = addTime(manager) + assert(secondAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) + clock.tick(100L) + onSchedulerBacklogged(manager) + assert(addTime(manager) === secondAddTime) // timer is already started + assert(addTime(manager) !== firstAddTime) + assert(firstAddTime !== secondAddTime) + } + + test("starting/canceling remove timers") { + val sc = createSparkContext(2, 10) + val clock = new TestClock(14444L) + val manager = sc.executorAllocationManager.get + manager.setClock(clock) + + // Starting remove timer is idempotent for each executor + assert(removeTimes(manager).isEmpty) + onExecutorIdle(manager, "1") + assert(removeTimes(manager).size === 1) + assert(removeTimes(manager).contains("1")) + val firstRemoveTime = removeTimes(manager)("1") + assert(firstRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000) + clock.tick(100L) + onExecutorIdle(manager, "1") + assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started + clock.tick(200L) + onExecutorIdle(manager, "1") + assert(removeTimes(manager)("1") === firstRemoveTime) + clock.tick(300L) + onExecutorIdle(manager, "2") + assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor + assert(removeTimes(manager)("2") === clock.getTimeMillis + executorIdleTimeout * 1000) + clock.tick(400L) + onExecutorIdle(manager, "3") + assert(removeTimes(manager)("3") !== firstRemoveTime) + assert(removeTimes(manager)("3") === clock.getTimeMillis + executorIdleTimeout * 1000) + assert(removeTimes(manager).size === 3) + assert(removeTimes(manager).contains("2")) + assert(removeTimes(manager).contains("3")) + + // Restart remove timer + clock.tick(1000L) + onExecutorBusy(manager, "1") + assert(removeTimes(manager).size === 2) + onExecutorIdle(manager, "1") + assert(removeTimes(manager).size === 3) + assert(removeTimes(manager).contains("1")) + val secondRemoveTime = removeTimes(manager)("1") + assert(secondRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000) + assert(removeTimes(manager)("1") === secondRemoveTime) // timer is already started + assert(removeTimes(manager)("1") !== firstRemoveTime) + assert(firstRemoveTime !== secondRemoveTime) + } + + test("mock polling loop with no events") { + val sc = createSparkContext(1, 20) + val manager = sc.executorAllocationManager.get + val clock = new TestClock(2020L) + manager.setClock(clock) + + // No events - we should not be adding or removing + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + schedule(manager) + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(100L) + schedule(manager) + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(1000L) + schedule(manager) + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(10000L) + schedule(manager) + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + } + + test("mock polling loop add behavior") { + val sc = createSparkContext(1, 20) + val clock = new TestClock(2020L) + val manager = sc.executorAllocationManager.get + manager.setClock(clock) + + // Scheduler queue backlogged + onSchedulerBacklogged(manager) + clock.tick(schedulerBacklogTimeout * 1000 / 2) + schedule(manager) + assert(numExecutorsPending(manager) === 0) // timer not exceeded yet + clock.tick(schedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 1) // first timer exceeded + clock.tick(sustainedSchedulerBacklogTimeout * 1000 / 2) + schedule(manager) + assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded + + // Scheduler queue drained + onSchedulerQueueEmpty(manager) + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7) // timer is canceled + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7) + + // Scheduler queue backlogged again + onSchedulerBacklogged(manager) + clock.tick(schedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7 + 1) // timer restarted + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7 + 1 + 2) + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4) + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 20) // limit reached + } + + test("mock polling loop remove behavior") { + val sc = createSparkContext(1, 20) + val clock = new TestClock(2020L) + val manager = sc.executorAllocationManager.get + manager.setClock(clock) + + // Remove idle executors on timeout + onExecutorAdded(manager, "executor-1") + onExecutorAdded(manager, "executor-2") + onExecutorAdded(manager, "executor-3") + assert(removeTimes(manager).size === 3) + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(executorIdleTimeout * 1000 / 2) + schedule(manager) + assert(removeTimes(manager).size === 3) // idle threshold not reached yet + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(executorIdleTimeout * 1000) + schedule(manager) + assert(removeTimes(manager).isEmpty) // idle threshold exceeded + assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining) + + // Mark a subset as busy - only idle executors should be removed + onExecutorAdded(manager, "executor-4") + onExecutorAdded(manager, "executor-5") + onExecutorAdded(manager, "executor-6") + onExecutorAdded(manager, "executor-7") + assert(removeTimes(manager).size === 5) // 5 active executors + assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed + onExecutorBusy(manager, "executor-4") + onExecutorBusy(manager, "executor-5") + onExecutorBusy(manager, "executor-6") // 3 busy and 2 idle (of the 5 active ones) + schedule(manager) + assert(removeTimes(manager).size === 2) // remove only idle executors + assert(!removeTimes(manager).contains("executor-4")) + assert(!removeTimes(manager).contains("executor-5")) + assert(!removeTimes(manager).contains("executor-6")) + assert(executorsPendingToRemove(manager).size === 2) + clock.tick(executorIdleTimeout * 1000) + schedule(manager) + assert(removeTimes(manager).isEmpty) // idle executors are removed + assert(executorsPendingToRemove(manager).size === 4) + assert(!executorsPendingToRemove(manager).contains("executor-4")) + assert(!executorsPendingToRemove(manager).contains("executor-5")) + assert(!executorsPendingToRemove(manager).contains("executor-6")) + + // Busy executors are now idle and should be removed + onExecutorIdle(manager, "executor-4") + onExecutorIdle(manager, "executor-5") + onExecutorIdle(manager, "executor-6") + schedule(manager) + assert(removeTimes(manager).size === 3) // 0 busy and 3 idle + assert(removeTimes(manager).contains("executor-4")) + assert(removeTimes(manager).contains("executor-5")) + assert(removeTimes(manager).contains("executor-6")) + assert(executorsPendingToRemove(manager).size === 4) + clock.tick(executorIdleTimeout * 1000) + schedule(manager) + assert(removeTimes(manager).isEmpty) + assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining) + } + + test("listeners trigger add executors correctly") { + val sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(addTime(manager) === NOT_SET) + + // Starting a stage should start the add timer + val numTasks = 10 + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, numTasks))) + assert(addTime(manager) !== NOT_SET) + + // Starting a subset of the tasks should not cancel the add timer + val taskInfos = (0 to numTasks - 1).map { i => createTaskInfo(i, i, "executor-1") } + taskInfos.tail.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, info)) } + assert(addTime(manager) !== NOT_SET) + + // Starting all remaining tasks should cancel the add timer + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, taskInfos.head)) + assert(addTime(manager) === NOT_SET) + + // Start two different stages + // The add timer should be canceled only if all tasks in both stages start running + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, numTasks))) + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, numTasks))) + assert(addTime(manager) !== NOT_SET) + taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, info)) } + assert(addTime(manager) !== NOT_SET) + taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, info)) } + assert(addTime(manager) === NOT_SET) + } + + test("listeners trigger remove executors correctly") { + val sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(removeTimes(manager).isEmpty) + + // Added executors should start the remove timers for each executor + (1 to 5).map("executor-" + _).foreach { id => onExecutorAdded(manager, id) } + assert(removeTimes(manager).size === 5) + + // Starting a task cancel the remove timer for that executor + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(1, 1, "executor-1"))) + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(2, 2, "executor-2"))) + assert(removeTimes(manager).size === 3) + assert(!removeTimes(manager).contains("executor-1")) + assert(!removeTimes(manager).contains("executor-2")) + + // Finishing all tasks running on an executor should start the remove timer for that executor + sc.listenerBus.postToAll(SparkListenerTaskEnd( + 0, 0, "task-type", Success, createTaskInfo(0, 0, "executor-1"), new TaskMetrics)) + sc.listenerBus.postToAll(SparkListenerTaskEnd( + 0, 0, "task-type", Success, createTaskInfo(2, 2, "executor-2"), new TaskMetrics)) + assert(removeTimes(manager).size === 4) + assert(!removeTimes(manager).contains("executor-1")) // executor-1 has not finished yet + assert(removeTimes(manager).contains("executor-2")) + sc.listenerBus.postToAll(SparkListenerTaskEnd( + 0, 0, "task-type", Success, createTaskInfo(1, 1, "executor-1"), new TaskMetrics)) + assert(removeTimes(manager).size === 5) + assert(removeTimes(manager).contains("executor-1")) // executor-1 has now finished + } + + test("listeners trigger add and remove executor callbacks correctly") { + val sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(executorIds(manager).isEmpty) + assert(removeTimes(manager).isEmpty) + + // New executors have registered + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + assert(executorIds(manager).size === 1) + assert(executorIds(manager).contains("executor-1")) + assert(removeTimes(manager).size === 1) + assert(removeTimes(manager).contains("executor-1")) + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-2", "host2", 1), 100L)) + assert(executorIds(manager).size === 2) + assert(executorIds(manager).contains("executor-2")) + assert(removeTimes(manager).size === 2) + assert(removeTimes(manager).contains("executor-2")) + + // Existing executors have disconnected + sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved( + 0L, BlockManagerId("executor-1", "host1", 1))) + assert(executorIds(manager).size === 1) + assert(!executorIds(manager).contains("executor-1")) + assert(removeTimes(manager).size === 1) + assert(!removeTimes(manager).contains("executor-1")) + + // Unknown executor has disconnected + sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved( + 0L, BlockManagerId("executor-3", "host3", 1))) + assert(executorIds(manager).size === 1) + assert(removeTimes(manager).size === 1) + } + +} + +/** + * Helper methods for testing ExecutorAllocationManager. + * This includes methods to access private methods and fields in ExecutorAllocationManager. + */ +private object ExecutorAllocationManagerSuite extends PrivateMethodTester { + private val schedulerBacklogTimeout = 1L + private val sustainedSchedulerBacklogTimeout = 2L + private val executorIdleTimeout = 3L + + private def createSparkContext(minExecutors: Int = 1, maxExecutors: Int = 5): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test-executor-allocation-manager") + .set("spark.dynamicAllocation.enabled", "true") + .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) + .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) + .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", + sustainedSchedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout.toString) + .set("spark.dynamicAllocation.testing", "true") + new SparkContext(conf) + } + + private def createStageInfo(stageId: Int, numTasks: Int): StageInfo = { + new StageInfo(stageId, 0, "name", numTasks, Seq.empty, "no details") + } + + private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = { + new TaskInfo(taskId, taskIndex, 0, 0, executorId, "", TaskLocality.ANY, speculative = false) + } + + /* ------------------------------------------------------- * + | Helper methods for accessing private methods and fields | + * ------------------------------------------------------- */ + + private val _numExecutorsToAdd = PrivateMethod[Int]('numExecutorsToAdd) + private val _numExecutorsPending = PrivateMethod[Int]('numExecutorsPending) + private val _executorsPendingToRemove = + PrivateMethod[collection.Set[String]]('executorsPendingToRemove) + private val _executorIds = PrivateMethod[collection.Set[String]]('executorIds) + private val _addTime = PrivateMethod[Long]('addTime) + private val _removeTimes = PrivateMethod[collection.Map[String, Long]]('removeTimes) + private val _schedule = PrivateMethod[Unit]('schedule) + private val _addExecutors = PrivateMethod[Int]('addExecutors) + private val _removeExecutor = PrivateMethod[Boolean]('removeExecutor) + private val _onExecutorAdded = PrivateMethod[Unit]('onExecutorAdded) + private val _onExecutorRemoved = PrivateMethod[Unit]('onExecutorRemoved) + private val _onSchedulerBacklogged = PrivateMethod[Unit]('onSchedulerBacklogged) + private val _onSchedulerQueueEmpty = PrivateMethod[Unit]('onSchedulerQueueEmpty) + private val _onExecutorIdle = PrivateMethod[Unit]('onExecutorIdle) + private val _onExecutorBusy = PrivateMethod[Unit]('onExecutorBusy) + + private def numExecutorsToAdd(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _numExecutorsToAdd() + } + + private def numExecutorsPending(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _numExecutorsPending() + } + + private def executorsPendingToRemove( + manager: ExecutorAllocationManager): collection.Set[String] = { + manager invokePrivate _executorsPendingToRemove() + } + + private def executorIds(manager: ExecutorAllocationManager): collection.Set[String] = { + manager invokePrivate _executorIds() + } + + private def addTime(manager: ExecutorAllocationManager): Long = { + manager invokePrivate _addTime() + } + + private def removeTimes(manager: ExecutorAllocationManager): collection.Map[String, Long] = { + manager invokePrivate _removeTimes() + } + + private def schedule(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _schedule() + } + + private def addExecutors(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _addExecutors() + } + + private def removeExecutor(manager: ExecutorAllocationManager, id: String): Boolean = { + manager invokePrivate _removeExecutor(id) + } + + private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { + manager invokePrivate _onExecutorAdded(id) + } + + private def onExecutorRemoved(manager: ExecutorAllocationManager, id: String): Unit = { + manager invokePrivate _onExecutorRemoved(id) + } + + private def onSchedulerBacklogged(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _onSchedulerBacklogged() + } + + private def onSchedulerQueueEmpty(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _onSchedulerQueueEmpty() + } + + private def onExecutorIdle(manager: ExecutorAllocationManager, id: String): Unit = { + manager invokePrivate _onExecutorIdle(id) + } + + private def onExecutorBusy(manager: ExecutorAllocationManager, id: String): Unit = { + manager invokePrivate _onExecutorBusy(id) + } +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 68073798886dd..e90672c004d4b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -505,7 +505,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, driver ! x case RequestExecutors(requestedTotal) => - logInfo(s"Driver requested a total number of executors of $requestedTotal.") + logInfo(s"Driver requested a total number of $requestedTotal executor(s).") Option(allocator) match { case Some(a) => a.requestTotalExecutors(requestedTotal) case None => logWarning("Container allocator is not ready to request executors yet.") From 1234258077b1f4050845e9fb73066b37f981c72a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 29 Oct 2014 17:59:16 -0700 Subject: [PATCH 081/115] [SPARK-4053][Streaming] Made the ReceiverSuite test more reliable, by fixing block generator throttling In the unit test that checked whether blocks generated by throttled block generator had expected number of records, the thresholds are too tight, which sometimes led to the test failing. This PR fixes it by relaxing the thresholds and the time intervals for testing. Author: Tathagata Das Closes #2900 from tdas/receiver-suite-flakiness and squashes the following commits: 28508a2 [Tathagata Das] Made the ReceiverSuite test more reliable --- ...eceiverSuite.scala => ReceiverSuite.scala} | 44 +++++++++++++------ 1 file changed, 30 insertions(+), 14 deletions(-) rename streaming/src/test/scala/org/apache/spark/streaming/{NetworkReceiverSuite.scala => ReceiverSuite.scala} (86%) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala similarity index 86% rename from streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index eb6e88cf5520d..0f6a9489dbe0d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -31,9 +31,9 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ /** Testsuite for testing the network receiver behavior */ -class NetworkReceiverSuite extends FunSuite with Timeouts { +class ReceiverSuite extends FunSuite with Timeouts { - test("network receiver life cycle") { + test("receiver life cycle") { val receiver = new FakeReceiver val executor = new FakeReceiverSupervisor(receiver) @@ -152,8 +152,8 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { test("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 50 - val maxRate = 200 + val blockInterval = 100 + val maxRate = 100 val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) @@ -175,19 +175,35 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { } blockGenerator.stop() - val recordedData = blockGeneratorListener.arrayBuffers - assert(blockGeneratorListener.arrayBuffers.size > 0) - assert(recordedData.flatten.toSet === generatedData.toSet) + val recordedBlocks = blockGeneratorListener.arrayBuffers + val recordedData = recordedBlocks.flatten + assert(blockGeneratorListener.arrayBuffers.size > 0, "No blocks received") + assert(recordedData.toSet === generatedData.toSet, "Received data not same") + // recordedData size should be close to the expected rate - assert(recordedData.flatten.size >= expectedMessages * 0.9 && - recordedData.flatten.size <= expectedMessages * 1.1 ) - // the first and last block may be incomplete, so we slice them out - recordedData.slice(1, recordedData.size - 1).foreach { block => - assert(block.size >= expectedMessagesPerBlock * 0.8 && - block.size <= expectedMessagesPerBlock * 1.2 ) - } + val minExpectedMessages = expectedMessages - 3 + val maxExpectedMessages = expectedMessages + 1 + val numMessages = recordedData.size + assert( + numMessages >= minExpectedMessages && numMessages <= maxExpectedMessages, + s"#records received = $numMessages, not between $minExpectedMessages and $maxExpectedMessages" + ) + + val minExpectedMessagesPerBlock = expectedMessagesPerBlock - 3 + val maxExpectedMessagesPerBlock = expectedMessagesPerBlock + 1 + val receivedBlockSizes = recordedBlocks.map { _.size }.mkString(",") + println(minExpectedMessagesPerBlock, maxExpectedMessagesPerBlock, ":", receivedBlockSizes) + assert( + // the first and last block may be incomplete, so we slice them out + recordedBlocks.drop(1).dropRight(1).forall { block => + block.size >= minExpectedMessagesPerBlock && block.size <= maxExpectedMessagesPerBlock + }, + s"# records in received blocks = [$receivedBlockSizes], not between " + + s"$minExpectedMessagesPerBlock and $maxExpectedMessagesPerBlock" + ) } + /** * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. */ From cd739bd756875bd52e9bd8ae801e0ae10a1f6937 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Wed, 29 Oct 2014 23:02:58 -0700 Subject: [PATCH 082/115] [SPARK-1720][SPARK-1719] use LD_LIBRARY_PATH instead of -Djava.library.path - [X] Standalone - [X] YARN - [X] Mesos - [X] Mac OS X - [X] Linux - [ ] Windows This is another implementation about #1031 Author: GuoQiang Li Closes #2711 from witgo/SPARK-1719 and squashes the following commits: c7b26f6 [GuoQiang Li] review commits 4488e41 [GuoQiang Li] Refactoring CommandUtils a444094 [GuoQiang Li] review commits 40c0b4a [GuoQiang Li] Add buildLocalCommand method c1a0ddd [GuoQiang Li] fix comments 156ce88 [GuoQiang Li] review commit 38aa377 [GuoQiang Li] Refactor CommandUtils.scala 4269e00 [GuoQiang Li] Refactor SparkSubmitDriverBootstrapper.scala 7a1d634 [GuoQiang Li] use LD_LIBRARY_PATH instead of -Djava.library.path --- bin/spark-class | 6 +- .../scala/org/apache/spark/SparkConf.scala | 13 ++++ .../SparkSubmitDriverBootstrapper.scala | 17 ++--- .../spark/deploy/worker/CommandUtils.scala | 68 ++++++++++++++++--- .../spark/deploy/worker/DriverRunner.scala | 23 ++----- .../spark/deploy/worker/ExecutorRunner.scala | 26 +++---- .../mesos/CoarseMesosSchedulerBackend.scala | 22 +++--- .../cluster/mesos/MesosSchedulerBackend.scala | 18 ++--- .../scala/org/apache/spark/util/Utils.scala | 42 +++++++++++- .../spark/deploy/CommandUtilsSuite.scala | 37 ++++++++++ .../deploy/worker/ExecutorRunnerTest.scala | 5 +- .../apache/spark/deploy/yarn/ClientBase.scala | 14 +++- .../deploy/yarn/ExecutorRunnableUtil.scala | 11 ++- 13 files changed, 221 insertions(+), 81 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala diff --git a/bin/spark-class b/bin/spark-class index 91d858bc063d0..925367b0dd187 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -81,7 +81,11 @@ case "$1" in OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} if [ -n "$SPARK_SUBMIT_LIBRARY_PATH" ]; then - OUR_JAVA_OPTS="$OUR_JAVA_OPTS -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" + if [[ $OSTYPE == darwin* ]]; then + export DYLD_LIBRARY_PATH="$SPARK_SUBMIT_LIBRARY_PATH:$DYLD_LIBRARY_PATH" + else + export LD_LIBRARY_PATH="$SPARK_SUBMIT_LIBRARY_PATH:$LD_LIBRARY_PATH" + fi fi if [ -n "$SPARK_SUBMIT_DRIVER_MEMORY" ]; then OUR_JAVA_MEM="$SPARK_SUBMIT_DRIVER_MEMORY" diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index dbbcc23305c50..ad0a9017afead 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -244,6 +244,19 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { val executorClasspathKey = "spark.executor.extraClassPath" val driverOptsKey = "spark.driver.extraJavaOptions" val driverClassPathKey = "spark.driver.extraClassPath" + val driverLibraryPathKey = "spark.driver.extraLibraryPath" + + // Used by Yarn in 1.1 and before + sys.props.get("spark.driver.libraryPath").foreach { value => + val warning = + s""" + |spark.driver.libraryPath was detected (set to '$value'). + |This is deprecated in Spark 1.2+. + | + |Please instead use: $driverLibraryPathKey + """.stripMargin + logWarning(warning) + } // Validate spark.executor.extraJavaOptions settings.get(executorOptsKey).map { javaOpts => diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index 0125330589da5..2b894a796c8c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -82,17 +82,8 @@ private[spark] object SparkSubmitDriverBootstrapper { .orElse(confDriverMemory) .getOrElse(defaultDriverMemory) - val newLibraryPath = - if (submitLibraryPath.isDefined) { - // SPARK_SUBMIT_LIBRARY_PATH is already captured in JAVA_OPTS - "" - } else { - confLibraryPath.map("-Djava.library.path=" + _).getOrElse("") - } - val newClasspath = if (submitClasspath.isDefined) { - // SPARK_SUBMIT_CLASSPATH is already captured in CLASSPATH classpath } else { classpath + confClasspath.map(sys.props("path.separator") + _).getOrElse("") @@ -114,7 +105,6 @@ private[spark] object SparkSubmitDriverBootstrapper { val command: Seq[String] = Seq(runner) ++ Seq("-cp", newClasspath) ++ - Seq(newLibraryPath) ++ filteredJavaOpts ++ Seq(s"-Xms$newDriverMemory", s"-Xmx$newDriverMemory") ++ Seq("org.apache.spark.deploy.SparkSubmit") ++ @@ -130,6 +120,13 @@ private[spark] object SparkSubmitDriverBootstrapper { // Start the driver JVM val filteredCommand = command.filter(_.nonEmpty) val builder = new ProcessBuilder(filteredCommand) + val env = builder.environment() + + if (submitLibraryPath.isEmpty && confLibraryPath.nonEmpty) { + val libraryPaths = confLibraryPath ++ sys.env.get(Utils.libraryPathEnvName) + env.put(Utils.libraryPathEnvName, libraryPaths.mkString(sys.props("path.separator"))) + } + val process = builder.start() // Redirect stdout and stderr from the child JVM diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 2e9be2a180c68..aba2e20118d7a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -20,6 +20,8 @@ package org.apache.spark.deploy.worker import java.io.{File, FileOutputStream, InputStream, IOException} import java.lang.System._ +import scala.collection.Map + import org.apache.spark.Logging import org.apache.spark.deploy.Command import org.apache.spark.util.Utils @@ -29,7 +31,29 @@ import org.apache.spark.util.Utils */ private[spark] object CommandUtils extends Logging { - def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { + + /** + * Build a ProcessBuilder based on the given parameters. + * The `env` argument is exposed for testing. + */ + def buildProcessBuilder( + command: Command, + memory: Int, + sparkHome: String, + substituteArguments: String => String, + classPaths: Seq[String] = Seq[String](), + env: Map[String, String] = sys.env): ProcessBuilder = { + val localCommand = buildLocalCommand(command, substituteArguments, classPaths, env) + val commandSeq = buildCommandSeq(localCommand, memory, sparkHome) + val builder = new ProcessBuilder(commandSeq: _*) + val environment = builder.environment() + for ((key, value) <- localCommand.environment) { + environment.put(key, value) + } + builder + } + + private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { val runner = sys.env.get("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() @@ -38,11 +62,41 @@ object CommandUtils extends Logging { command.arguments } + /** + * Build a command based on the given one, taking into account the local environment + * of where this command is expected to run, substitute any placeholders, and append + * any extra class paths. + */ + private def buildLocalCommand( + command: Command, + substituteArguments: String => String, + classPath: Seq[String] = Seq[String](), + env: Map[String, String]): Command = { + val libraryPathName = Utils.libraryPathEnvName + val libraryPathEntries = command.libraryPathEntries + val cmdLibraryPath = command.environment.get(libraryPathName) + + val newEnvironment = if (libraryPathEntries.nonEmpty && libraryPathName.nonEmpty) { + val libraryPaths = libraryPathEntries ++ cmdLibraryPath ++ env.get(libraryPathName) + command.environment + ((libraryPathName, libraryPaths.mkString(File.pathSeparator))) + } else { + command.environment + } + + Command( + command.mainClass, + command.arguments.map(substituteArguments), + newEnvironment, + command.classPathEntries ++ classPath, + Seq[String](), // library path already captured in environment variable + command.javaOpts) + } + /** * Attention: this must always be aligned with the environment variables in the run scripts and * the way the JAVA_OPTS are assembled there. */ - def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { + private def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") // Exists for backwards compatibility with older Spark versions @@ -53,14 +107,6 @@ object CommandUtils extends Logging { logWarning("Set SPARK_LOCAL_DIRS for node-specific storage locations.") } - val libraryOpts = - if (command.libraryPathEntries.size > 0) { - val joined = command.libraryPathEntries.mkString(File.pathSeparator) - Seq(s"-Djava.library.path=$joined") - } else { - Seq() - } - // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( @@ -71,7 +117,7 @@ object CommandUtils extends Logging { val javaVersion = System.getProperty("java.version") val permGenOpt = if (!javaVersion.startsWith("1.8")) Some("-XX:MaxPermSize=128m") else None Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - permGenOpt ++ libraryOpts ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts + permGenOpt ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 3bf0b9492df0a..28cab36c7b9e2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -76,17 +76,9 @@ private[spark] class DriverRunner( // Make sure user application jar is on the classpath // TODO: If we add ability to submit multiple jars they should also be added here - val classPath = driverDesc.command.classPathEntries ++ Seq(s"$localJarFilename") - val newCommand = Command( - driverDesc.command.mainClass, - driverDesc.command.arguments.map(substituteVariables), - driverDesc.command.environment, - classPath, - driverDesc.command.libraryPathEntries, - driverDesc.command.javaOpts) - val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, - sparkHome.getAbsolutePath) - launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) + val builder = CommandUtils.buildProcessBuilder(driverDesc.command, driverDesc.mem, + sparkHome.getAbsolutePath, substituteVariables, Seq(localJarFilename)) + launchDriver(builder, driverDir, driverDesc.supervise) } catch { case e: Exception => finalException = Some(e) @@ -165,11 +157,8 @@ private[spark] class DriverRunner( localJarFilename } - private def launchDriver(command: Seq[String], envVars: Map[String, String], baseDir: File, - supervise: Boolean) { - val builder = new ProcessBuilder(command: _*).directory(baseDir) - envVars.map{ case(k,v) => builder.environment().put(k, v) } - + private def launchDriver(builder: ProcessBuilder, baseDir: File, supervise: Boolean) { + builder.directory(baseDir) def initialize(process: Process) = { // Redirect stdout and stderr to files val stdout = new File(baseDir, "stdout") @@ -177,7 +166,7 @@ private[spark] class DriverRunner( val stderr = new File(baseDir, "stderr") val header = "Launch Command: %s\n%s\n\n".format( - command.mkString("\"", "\" \"", "\""), "=" * 40) + builder.command.mkString("\"", "\" \"", "\""), "=" * 40) Files.append(header, stderr, UTF_8) CommandUtils.redirectStream(process.getErrorStream, stderr) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 030a651469b64..8ba6a01bbcb97 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.worker import java.io._ +import scala.collection.JavaConversions._ + import akka.actor.ActorRef import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files @@ -115,33 +117,21 @@ private[spark] class ExecutorRunner( case other => other } - def getCommandSeq = { - val command = Command( - appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables), - appDesc.command.environment, - appDesc.command.classPathEntries, - appDesc.command.libraryPathEntries, - appDesc.command.javaOpts) - CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) - } - /** * Download and run the executor described in our ApplicationDescription */ def fetchAndRunExecutor() { try { // Launch the process - val command = getCommandSeq + val builder = CommandUtils.buildProcessBuilder(appDesc.command, memory, + sparkHome.getAbsolutePath, substituteVariables) + val command = builder.command() logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) - val builder = new ProcessBuilder(command: _*).directory(executorDir) - val env = builder.environment() - for ((key, value) <- appDesc.command.environment) { - env.put(key, value) - } + + builder.directory(executorDir) // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command - env.put("SPARK_LAUNCH_WITH_SCALA", "0") + builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0") process = builder.start() val header = "Spark Executor Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d7f88de4b40aa..d8c0e2f66df01 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -31,6 +31,7 @@ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTas import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -120,16 +121,18 @@ private[spark] class CoarseMesosSchedulerBackend( environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions") + val extraJavaOpts = conf.get("spark.executor.extraJavaOptions", "") - val libraryPathOption = "spark.executor.extraLibraryPath" - val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") - val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + // Set the environment variable through a command prefix + // to append to the existing value of the variable + val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => + Utils.libraryPathEnvPrefix(Seq(p)) + }.getOrElse("") environment.addVariables( Environment.Variable.newBuilder() .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraOpts) + .setValue(extraJavaOpts) .build()) sc.executorEnvs.foreach { case (key, value) => @@ -150,16 +153,17 @@ private[spark] class CoarseMesosSchedulerBackend( if (uri == null) { val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores, appId)) + "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s".format( + prefixEnv, runScript, driverUrl, offer.getSlaveId.getValue, + offer.getHostname, numCores, appId)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( - ("cd %s*; " + + ("cd %s*; %s " + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s") - .format(basename, driverUrl, offer.getSlaveId.getValue, + .format(basename, prefixEnv, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores, appId)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index e0f2fd622f54c..8e2faff90f9b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -98,15 +98,16 @@ private[spark] class MesosSchedulerBackend( environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") - val extraLibraryPath = sc.conf.getOption("spark.executor.extraLibraryPath").map { lp => - s"-Djava.library.path=$lp" - } - val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").getOrElse("") + + val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => + Utils.libraryPathEnvPrefix(Seq(p)) + }.getOrElse("") + environment.addVariables( Environment.Variable.newBuilder() .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraOpts) + .setValue(extraJavaOpts) .build()) sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() @@ -118,12 +119,13 @@ private[spark] class MesosSchedulerBackend( .setEnvironment(environment) val uri = sc.conf.get("spark.executor.uri", null) if (uri == null) { - command.setValue(new File(executorSparkHome, "/sbin/spark-executor").getCanonicalPath) + val executorPath = new File(executorSparkHome, "/sbin/spark-executor").getCanonicalPath + command.setValue("%s %s".format(prefixEnv, executorPath)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head - command.setValue("cd %s*; ./sbin/spark-executor".format(basename)) + command.setValue("cd %s*; %s ./sbin/spark-executor".format(basename, prefixEnv)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } val cpus = Resource.newBuilder() 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 0daab91143e47..063895d3c548d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -23,8 +23,6 @@ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} -import org.eclipse.jetty.util.MultiException - import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -39,6 +37,7 @@ import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.log4j.PropertyConfigurator import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.eclipse.jetty.util.MultiException import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} @@ -1381,6 +1380,11 @@ private[spark] object Utils extends Logging { */ val isWindows = SystemUtils.IS_OS_WINDOWS + /** + * Whether the underlying operating system is Mac OS X. + */ + val isMac = SystemUtils.IS_OS_MAC_OSX + /** * Pattern for matching a Windows drive, which contains only a single alphabet character. */ @@ -1714,6 +1718,40 @@ private[spark] object Utils extends Logging { method.invoke(obj, values.toSeq: _*) } + /** + * Return the current system LD_LIBRARY_PATH name + */ + def libraryPathEnvName: String = { + if (isWindows) { + "PATH" + } else if (isMac) { + "DYLD_LIBRARY_PATH" + } else { + "LD_LIBRARY_PATH" + } + } + + /** + * Return the prefix of a command that appends the given library paths to the + * system-specific library path environment variable. On Unix, for instance, + * this returns the string LD_LIBRARY_PATH="path1:path2:$LD_LIBRARY_PATH". + */ + def libraryPathEnvPrefix(libraryPaths: Seq[String]): String = { + val libraryPathScriptVar = if (isWindows) { + s"%${libraryPathEnvName}%" + } else { + "$" + libraryPathEnvName + } + val libraryPath = (libraryPaths :+ libraryPathScriptVar).mkString("\"", + File.pathSeparator, "\"") + val ampersand = if (Utils.isWindows) { + " &" + } else { + "" + } + s"$libraryPathEnvName=$libraryPath$ampersand" + } + } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala new file mode 100644 index 0000000000000..7915ee75d8778 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.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.deploy + +import org.apache.spark.deploy.worker.CommandUtils +import org.apache.spark.util.Utils + +import org.scalatest.{FunSuite, Matchers} + +class CommandUtilsSuite extends FunSuite with Matchers { + + test("set libraryPath correctly") { + val appId = "12345-worker321-9876" + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) + val cmd = new Command("mainClass", Seq(), Map(), Seq(), Seq("libraryPathToB"), Seq()) + val builder = CommandUtils.buildProcessBuilder(cmd, 512, sparkHome, t => t) + val libraryPath = Utils.libraryPathEnvName + val env = builder.environment + env.keySet should contain(libraryPath) + assert(env.get(libraryPath).startsWith("libraryPathToB")) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 5e2592e8d2e8d..196217062991e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.worker import java.io.File +import scala.collection.JavaConversions._ + import org.scalatest.FunSuite import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} @@ -32,6 +34,7 @@ class ExecutorRunnerTest extends FunSuite { Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", new File(sparkHome), new File("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) - assert(er.getCommandSeq.last === appId) + val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) + assert(builder.command().last === appId) } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8ea0e7cf40a14..f95d72379171c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} +import org.apache.spark.util.Utils /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. @@ -312,6 +313,10 @@ private[spark] trait ClientBase extends Logging { val javaOpts = ListBuffer[String]() + // Set the environment variable through a command prefix + // to append to the existing value of the variable + var prefixEnv: Option[String] = None + // Add Xmx for AM memory javaOpts += "-Xmx" + args.amMemory + "m" @@ -348,8 +353,11 @@ private[spark] trait ClientBase extends Logging { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) .foreach(opts => javaOpts += opts) - sparkConf.getOption("spark.driver.libraryPath") - .foreach(p => javaOpts += s"-Djava.library.path=$p") + val libraryPaths = Seq(sys.props.get("spark.driver.extraLibraryPath"), + sys.props.get("spark.driver.libraryPath")).flatten + if (libraryPaths.nonEmpty) { + prefixEnv = Some(Utils.libraryPathEnvPrefix(libraryPaths)) + } } // For log4j configuration to reference @@ -384,7 +392,7 @@ private[spark] trait ClientBase extends Logging { "--num-executors ", args.numExecutors.toString) // Command for the ApplicationMaster - val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ + val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ javaOpts ++ amArgs ++ Seq( "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 5cb4753de2e84..88dad0febd03f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.Utils trait ExecutorRunnableUtil extends Logging { @@ -47,6 +48,11 @@ trait ExecutorRunnableUtil extends Logging { localResources: HashMap[String, LocalResource]): List[String] = { // Extra options for the JVM val javaOpts = ListBuffer[String]() + + // Set the environment variable through a command prefix + // to append to the existing value of the variable + var prefixEnv: Option[String] = None + // Set the JVM memory val executorMemoryString = executorMemory + "m" javaOpts += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " @@ -58,6 +64,9 @@ trait ExecutorRunnableUtil extends Logging { sys.env.get("SPARK_JAVA_OPTS").foreach { opts => javaOpts += opts } + sys.props.get("spark.executor.extraLibraryPath").foreach { p => + prefixEnv = Some(Utils.libraryPathEnvPrefix(Seq(p))) + } javaOpts += "-Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) @@ -101,7 +110,7 @@ trait ExecutorRunnableUtil extends Logging { // For log4j configuration to reference javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) - val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", + val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server", // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in From 6db3157464e36f7a572ada5f1e7f88730aa23dbd Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 29 Oct 2014 23:52:46 -0700 Subject: [PATCH 083/115] [SPARK-4102] Remove unused ShuffleReader.stop() method. This method is not implemented by the only subclass (HashShuffleReader), nor is it ever called. While the use of Scala's fancy "???" was pretty exciting, the method's existence can only lead to confusion and it therefore should be deleted. mateiz was there a reason for adding this that I'm missing? Author: Kay Ousterhout Closes #2966 from kayousterhout/SPARK-4102 and squashes the following commits: 532c564 [Kay Ousterhout] Added back commented-out method, as per Matei's request 904655e [Kay Ousterhout] [SPARK-4102] Remove unused ShuffleReader.stop() method. --- .../scala/org/apache/spark/shuffle/ShuffleReader.scala | 8 ++++++-- .../org/apache/spark/shuffle/hash/HashShuffleReader.scala | 3 --- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala index b30e366d06006..292e48314ee10 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala @@ -24,6 +24,10 @@ private[spark] trait ShuffleReader[K, C] { /** Read the combined key-values for this reduce task */ def read(): Iterator[Product2[K, C]] - /** Close this reader */ - def stop(): Unit + /** + * Close this reader. + * TODO: Add this back when we make the ShuffleReader a developer API that others can implement + * (at which point this will likely be necessary). + */ + // def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 88a5f1e5ddf58..5baf45db45c17 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -66,7 +66,4 @@ private[spark] class HashShuffleReader[K, C]( aggregatedIter } } - - /** Close this reader */ - override def stop(): Unit = ??? } From c7ad0852084dc28f3ebc144adfd4928b23f1c8ea Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 30 Oct 2014 00:05:57 -0700 Subject: [PATCH 084/115] [SPARK-4130][MLlib] Fixing libSVM parser bug with extra whitespace This simple patch filters out extra whitespace entries. Author: Joseph E. Gonzalez Author: Joey Closes #2996 from jegonzal/loadLibSVM and squashes the following commits: e0227ab [Joey] improving readability e028e84 [Joseph E. Gonzalez] fixing whitespace bug in loadLibSVMFile when parsing libSVM files --- mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index dce0adffa6249..b88e08bf148ae 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -76,7 +76,7 @@ object MLUtils { .map { line => val items = line.split(' ') val label = items.head.toDouble - val (indices, values) = items.tail.map { item => + val (indices, values) = items.tail.filter(_.nonEmpty).map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. val value = indexAndValue(1).toDouble From d9327192eee7f18e92381c59a42b0e1770f1f8f4 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 30 Oct 2014 12:00:56 -0700 Subject: [PATCH 085/115] SPARK-4111 [MLlib] add regression metrics Add RegressionMetrics.scala as regression metrics used for evaluation and corresponding test case RegressionMetricsSuite.scala. Author: Yanbo Liang Author: liangyanbo Closes #2978 from yanbohappy/regression_metrics and squashes the following commits: 730d0a9 [Yanbo Liang] more clearly annotation 3d0bec1 [Yanbo Liang] rename and keep code style a8ad3e3 [Yanbo Liang] simplify code for keeping style d454909 [Yanbo Liang] rename parameter and function names, delete unused columns, add reference 2e56282 [liangyanbo] rename r2_score() and remove unused column 43bb12b [liangyanbo] add regression metrics --- .../mllib/evaluation/RegressionMetrics.scala | 89 +++++++++++++++++++ .../evaluation/RegressionMetricsSuite.scala | 52 +++++++++++ 2 files changed, 141 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala new file mode 100644 index 0000000000000..693117d820580 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.apache.spark.annotation.Experimental +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, MultivariateOnlineSummarizer} + +/** + * :: Experimental :: + * Evaluator for regression. + * + * @param predictionAndObservations an RDD of (prediction, observation) pairs. + */ +@Experimental +class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extends Logging { + + /** + * Use MultivariateOnlineSummarizer to calculate summary statistics of observations and errors. + */ + private lazy val summary: MultivariateStatisticalSummary = { + val summary: MultivariateStatisticalSummary = predictionAndObservations.map { + case (prediction, observation) => Vectors.dense(observation, observation - prediction) + }.aggregate(new MultivariateOnlineSummarizer())( + (summary, v) => summary.add(v), + (sum1, sum2) => sum1.merge(sum2) + ) + summary + } + + /** + * Returns the explained variance regression score. + * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) + * Reference: [[http://en.wikipedia.org/wiki/Explained_variation]] + */ + def explainedVariance: Double = { + 1 - summary.variance(1) / summary.variance(0) + } + + /** + * Returns the mean absolute error, which is a risk function corresponding to the + * expected value of the absolute error loss or l1-norm loss. + */ + def meanAbsoluteError: Double = { + summary.normL1(1) / summary.count + } + + /** + * Returns the mean squared error, which is a risk function corresponding to the + * expected value of the squared error loss or quadratic loss. + */ + def meanSquaredError: Double = { + val rmse = summary.normL2(1) / math.sqrt(summary.count) + rmse * rmse + } + + /** + * Returns the root mean squared error, which is defined as the square root of + * the mean squared error. + */ + def rootMeanSquaredError: Double = { + summary.normL2(1) / math.sqrt(summary.count) + } + + /** + * Returns R^2^, the coefficient of determination. + * Reference: [[http://en.wikipedia.org/wiki/Coefficient_of_determination]] + */ + def r2: Double = { + 1 - math.pow(summary.normL2(1), 2) / (summary.variance(0) * (summary.count - 1)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala new file mode 100644 index 0000000000000..5396d7b2b74fa --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.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.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class RegressionMetricsSuite extends FunSuite with LocalSparkContext { + + test("regression metrics") { + val predictionAndObservations = sc.parallelize( + Seq((2.5,3.0),(0.0,-0.5),(2.0,2.0),(8.0,7.0)), 2) + val metrics = new RegressionMetrics(predictionAndObservations) + assert(metrics.explainedVariance ~== 0.95717 absTol 1E-5, + "explained variance regression score mismatch") + assert(metrics.meanAbsoluteError ~== 0.5 absTol 1E-5, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.375 absTol 1E-5, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.61237 absTol 1E-5, + "root mean squared error mismatch") + assert(metrics.r2 ~== 0.94861 absTol 1E-5, "r2 score mismatch") + } + + test("regression metrics with complete fitting") { + val predictionAndObservations = sc.parallelize( + Seq((3.0,3.0),(0.0,0.0),(2.0,2.0),(8.0,8.0)), 2) + val metrics = new RegressionMetrics(predictionAndObservations) + assert(metrics.explainedVariance ~== 1.0 absTol 1E-5, + "explained variance regression score mismatch") + assert(metrics.meanAbsoluteError ~== 0.0 absTol 1E-5, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.0 absTol 1E-5, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.0 absTol 1E-5, + "root mean squared error mismatch") + assert(metrics.r2 ~== 1.0 absTol 1E-5, "r2 score mismatch") + } +} From 234de9232bcfa212317a8073c4a82c3863b36b14 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 30 Oct 2014 14:51:13 -0700 Subject: [PATCH 086/115] [SPARK-4028][Streaming] ReceivedBlockHandler interface to abstract the functionality of storage of received data As part of the initiative to prevent data loss on streaming driver failure, this JIRA tracks the subtask of implementing a ReceivedBlockHandler, that abstracts the functionality of storage of received data blocks. The default implementation will maintain the current behavior of storing the data into BlockManager. The optional implementation will store the data to both BlockManager as well as a write ahead log. Author: Tathagata Das Closes #2940 from tdas/driver-ha-rbh and squashes the following commits: 78a4aaa [Tathagata Das] Fixed bug causing test failures. f192f47 [Tathagata Das] Fixed import order. df5f320 [Tathagata Das] Updated code to use ReceivedBlockStoreResult as the return type for handler's storeBlock 33c30c9 [Tathagata Das] Added license, and organized imports. 2f025b3 [Tathagata Das] Updates based on PR comments. 18aec1e [Tathagata Das] Moved ReceivedBlockInfo back into spark.streaming.scheduler package 95a4987 [Tathagata Das] Added ReceivedBlockHandler and its associated tests --- .../dstream/ReceiverInputDStream.scala | 7 +- .../streaming/receiver/ReceivedBlock.scala | 35 +++ .../receiver/ReceivedBlockHandler.scala | 193 +++++++++++++ .../receiver/ReceiverSupervisorImpl.scala | 88 +++--- .../spark/streaming/scheduler/BatchInfo.scala | 2 +- .../spark/streaming/scheduler/JobSet.scala | 3 +- .../scheduler/ReceivedBlockInfo.scala | 28 ++ .../streaming/scheduler/ReceiverTracker.scala | 24 +- .../util/WriteAheadLogRandomReader.scala | 1 - .../streaming/ReceivedBlockHandlerSuite.scala | 258 ++++++++++++++++++ .../streaming/util/WriteAheadLogSuite.scala | 34 ++- 11 files changed, 603 insertions(+), 70 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index 391e40924f38a..bb47d373de63d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -23,8 +23,9 @@ import scala.reflect.ClassTag import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.BlockId import org.apache.spark.streaming._ -import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.receiver.{WriteAheadLogBasedStoreResult, BlockManagerBasedStoreResult, Receiver} import org.apache.spark.streaming.scheduler.ReceivedBlockInfo +import org.apache.spark.SparkException /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -65,10 +66,10 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont if (validTime >= graph.startTime) { val blockInfo = ssc.scheduler.receiverTracker.getReceivedBlockInfo(id) receivedBlockInfo(validTime) = blockInfo - val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) + val blockIds = blockInfo.map { _.blockStoreResult.blockId.asInstanceOf[BlockId] } Some(new BlockRDD[T](ssc.sc, blockIds)) } else { - Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) + Some(new BlockRDD[T](ssc.sc, Array.empty)) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala new file mode 100644 index 0000000000000..47968afef2dbf --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.language.existentials + +/** Trait representing a received block */ +private[streaming] sealed trait ReceivedBlock + +/** class representing a block received as an ArrayBuffer */ +private[streaming] case class ArrayBufferBlock(arrayBuffer: ArrayBuffer[_]) extends ReceivedBlock + +/** class representing a block received as an Iterator */ +private[streaming] case class IteratorBlock(iterator: Iterator[_]) extends ReceivedBlock + +/** class representing a block received as an ByteBuffer */ +private[streaming] case class ByteBufferBlock(byteBuffer: ByteBuffer) extends ReceivedBlock 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 new file mode 100644 index 0000000000000..fdf995320beb4 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.duration._ +import scala.language.{existentials, postfixOps} + +import WriteAheadLogBasedBlockHandler._ +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.storage._ +import org.apache.spark.streaming.util.{Clock, SystemClock, WriteAheadLogFileSegment, WriteAheadLogManager} +import org.apache.spark.util.Utils + +/** Trait that represents the metadata related to storage of blocks */ +private[streaming] trait ReceivedBlockStoreResult { + def blockId: StreamBlockId // Any implementation of this trait will store a block id +} + +/** Trait that represents a class that handles the storage of blocks received by receiver */ +private[streaming] trait ReceivedBlockHandler { + + /** Store a received block with the given block id and return related metadata */ + def storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock): ReceivedBlockStoreResult + + /** Cleanup old blocks older than the given threshold time */ + def cleanupOldBlock(threshTime: Long) +} + + +/** + * Implementation of [[org.apache.spark.streaming.receiver.ReceivedBlockStoreResult]] + * that stores the metadata related to storage of blocks using + * [[org.apache.spark.streaming.receiver.BlockManagerBasedBlockHandler]] + */ +private[streaming] case class BlockManagerBasedStoreResult(blockId: StreamBlockId) + extends ReceivedBlockStoreResult + + +/** + * Implementation of a [[org.apache.spark.streaming.receiver.ReceivedBlockHandler]] which + * stores the received blocks into a block manager with the specified storage level. + */ +private[streaming] class BlockManagerBasedBlockHandler( + blockManager: BlockManager, storageLevel: StorageLevel) + extends ReceivedBlockHandler with Logging { + + def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = { + val putResult: Seq[(BlockId, BlockStatus)] = block match { + case ArrayBufferBlock(arrayBuffer) => + blockManager.putIterator(blockId, arrayBuffer.iterator, storageLevel, tellMaster = true) + case IteratorBlock(iterator) => + blockManager.putIterator(blockId, iterator, storageLevel, tellMaster = true) + case ByteBufferBlock(byteBuffer) => + blockManager.putBytes(blockId, byteBuffer, storageLevel, tellMaster = true) + case o => + throw new SparkException( + s"Could not store $blockId to block manager, unexpected block type ${o.getClass.getName}") + } + if (!putResult.map { _._1 }.contains(blockId)) { + throw new SparkException( + s"Could not store $blockId to block manager with storage level $storageLevel") + } + BlockManagerBasedStoreResult(blockId) + } + + def cleanupOldBlock(threshTime: Long) { + // this is not used as blocks inserted into the BlockManager are cleared by DStream's clearing + // of BlockRDDs. + } +} + + +/** + * Implementation of [[org.apache.spark.streaming.receiver.ReceivedBlockStoreResult]] + * that stores the metadata related to storage of blocks using + * [[org.apache.spark.streaming.receiver.WriteAheadLogBasedBlockHandler]] + */ +private[streaming] case class WriteAheadLogBasedStoreResult( + blockId: StreamBlockId, + segment: WriteAheadLogFileSegment + ) extends ReceivedBlockStoreResult + + +/** + * Implementation of a [[org.apache.spark.streaming.receiver.ReceivedBlockHandler]] which + * stores the received blocks in both, a write ahead log and a block manager. + */ +private[streaming] class WriteAheadLogBasedBlockHandler( + blockManager: BlockManager, + streamId: Int, + storageLevel: StorageLevel, + conf: SparkConf, + hadoopConf: Configuration, + checkpointDir: String, + clock: Clock = new SystemClock + ) extends ReceivedBlockHandler with Logging { + + private val blockStoreTimeout = conf.getInt( + "spark.streaming.receiver.blockStoreTimeout", 30).seconds + private val rollingInterval = conf.getInt( + "spark.streaming.receiver.writeAheadLog.rollingInterval", 60) + private val maxFailures = conf.getInt( + "spark.streaming.receiver.writeAheadLog.maxFailures", 3) + + // Manages rolling log files + private val logManager = new WriteAheadLogManager( + checkpointDirToLogDir(checkpointDir, streamId), + hadoopConf, rollingInterval, maxFailures, + callerName = this.getClass.getSimpleName, + clock = clock + ) + + // For processing futures used in parallel block storing into block manager and write ahead log + // # threads = 2, so that both writing to BM and WAL can proceed in parallel + implicit private val executionContext = ExecutionContext.fromExecutorService( + Utils.newDaemonFixedThreadPool(2, this.getClass.getSimpleName)) + + /** + * This implementation stores the block into the block manager as well as a write ahead log. + * It does this in parallel, using Scala Futures, and returns only after the block has + * been stored in both places. + */ + def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = { + + // Serialize the block so that it can be inserted into both + val serializedBlock = block match { + case ArrayBufferBlock(arrayBuffer) => + blockManager.dataSerialize(blockId, arrayBuffer.iterator) + case IteratorBlock(iterator) => + blockManager.dataSerialize(blockId, iterator) + case ByteBufferBlock(byteBuffer) => + byteBuffer + case _ => + throw new Exception(s"Could not push $blockId to block manager, unexpected block type") + } + + // Store the block in block manager + val storeInBlockManagerFuture = Future { + val putResult = + blockManager.putBytes(blockId, serializedBlock, storageLevel, tellMaster = true) + if (!putResult.map { _._1 }.contains(blockId)) { + throw new SparkException( + s"Could not store $blockId to block manager with storage level $storageLevel") + } + } + + // Store the block in write ahead log + val storeInWriteAheadLogFuture = Future { + logManager.writeToLog(serializedBlock) + } + + // Combine the futures, wait for both to complete, and return the write ahead log segment + val combinedFuture = for { + _ <- storeInBlockManagerFuture + fileSegment <- storeInWriteAheadLogFuture + } yield fileSegment + val segment = Await.result(combinedFuture, blockStoreTimeout) + WriteAheadLogBasedStoreResult(blockId, segment) + } + + def cleanupOldBlock(threshTime: Long) { + logManager.cleanupOldLogs(threshTime) + } + + def stop() { + logManager.stop() + } +} + +private[streaming] object WriteAheadLogBasedBlockHandler { + def checkpointDirToLogDir(checkpointDir: String, streamId: Int): String = { + new Path(checkpointDir, new Path("receivedData", streamId.toString)).toString + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 53a3e6200e340..5360412330d37 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -25,16 +25,13 @@ import scala.concurrent.Await import akka.actor.{Actor, Props} import akka.pattern.ask - import com.google.common.base.Throwables - -import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.streaming.scheduler._ -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.hadoop.conf.Configuration +import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.storage.StreamBlockId -import org.apache.spark.streaming.scheduler.DeregisterReceiver -import org.apache.spark.streaming.scheduler.AddBlock -import org.apache.spark.streaming.scheduler.RegisterReceiver +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.util.WriteAheadLogFileSegment +import org.apache.spark.util.{AkkaUtils, Utils} /** * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] @@ -44,12 +41,26 @@ import org.apache.spark.streaming.scheduler.RegisterReceiver */ private[streaming] class ReceiverSupervisorImpl( receiver: Receiver[_], - env: SparkEnv + env: SparkEnv, + hadoopConf: Configuration, + checkpointDirOption: Option[String] ) extends ReceiverSupervisor(receiver, env.conf) with Logging { - private val blockManager = env.blockManager + private val receivedBlockHandler: ReceivedBlockHandler = { + if (env.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false)) { + if (checkpointDirOption.isEmpty) { + throw new SparkException( + "Cannot enable receiver write-ahead log without checkpoint directory set. " + + "Please use streamingContext.checkpoint() to set the checkpoint directory. " + + "See documentation for more details.") + } + new WriteAheadLogBasedBlockHandler(env.blockManager, receiver.streamId, + receiver.storageLevel, env.conf, hadoopConf, checkpointDirOption.get) + } else { + new BlockManagerBasedBlockHandler(env.blockManager, receiver.storageLevel) + } + } - private val storageLevel = receiver.storageLevel /** Remote Akka actor for the ReceiverTracker */ private val trackerActor = { @@ -105,47 +116,50 @@ private[streaming] class ReceiverSupervisorImpl( /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ def pushArrayBuffer( arrayBuffer: ArrayBuffer[_], - optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] + metadataOption: Option[Any], + blockIdOption: Option[StreamBlockId] ) { - val blockId = optionalBlockId.getOrElse(nextBlockId) - val time = System.currentTimeMillis - blockManager.putArray(blockId, arrayBuffer.toArray[Any], storageLevel, tellMaster = true) - logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") - reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) + pushAndReportBlock(ArrayBufferBlock(arrayBuffer), metadataOption, blockIdOption) } /** Store a iterator of received data as a data block into Spark's memory. */ def pushIterator( iterator: Iterator[_], - optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] + metadataOption: Option[Any], + blockIdOption: Option[StreamBlockId] ) { - val blockId = optionalBlockId.getOrElse(nextBlockId) - val time = System.currentTimeMillis - blockManager.putIterator(blockId, iterator, storageLevel, tellMaster = true) - logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") - reportPushedBlock(blockId, -1, optionalMetadata) + pushAndReportBlock(IteratorBlock(iterator), metadataOption, blockIdOption) } /** Store the bytes of received data as a data block into Spark's memory. */ def pushBytes( bytes: ByteBuffer, - optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] + metadataOption: Option[Any], + blockIdOption: Option[StreamBlockId] ) { - val blockId = optionalBlockId.getOrElse(nextBlockId) - val time = System.currentTimeMillis - blockManager.putBytes(blockId, bytes, storageLevel, tellMaster = true) - logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") - reportPushedBlock(blockId, -1, optionalMetadata) + pushAndReportBlock(ByteBufferBlock(bytes), metadataOption, blockIdOption) } - /** Report pushed block */ - def reportPushedBlock(blockId: StreamBlockId, numRecords: Long, optionalMetadata: Option[Any]) { - val blockInfo = ReceivedBlockInfo(streamId, blockId, numRecords, optionalMetadata.orNull) - trackerActor ! AddBlock(blockInfo) - logDebug("Reported block " + blockId) + /** Store block and report it to driver */ + def pushAndReportBlock( + receivedBlock: ReceivedBlock, + metadataOption: Option[Any], + blockIdOption: Option[StreamBlockId] + ) { + val blockId = blockIdOption.getOrElse(nextBlockId) + val numRecords = receivedBlock match { + case ArrayBufferBlock(arrayBuffer) => arrayBuffer.size + case _ => -1 + } + + val time = System.currentTimeMillis + val blockStoreResult = receivedBlockHandler.storeBlock(blockId, receivedBlock) + logDebug(s"Pushed block $blockId in ${(System.currentTimeMillis - time)} ms") + + val blockInfo = ReceivedBlockInfo(streamId, numRecords, blockStoreResult) + val future = trackerActor.ask(AddBlock(blockInfo))(askTimeout) + Await.result(future, askTimeout) + logDebug(s"Reported block $blockId") } /** Report error to the receiver tracker */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index a68aecb881117..92dc113f397ca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -17,8 +17,8 @@ package org.apache.spark.streaming.scheduler -import org.apache.spark.streaming.Time import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.streaming.Time /** * :: DeveloperApi :: diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index a69d74362173e..8c15a75b1b0e0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.scheduler -import scala.collection.mutable.{ArrayBuffer, HashSet} +import scala.collection.mutable.HashSet + import org.apache.spark.streaming.Time /** Class representing a set of Jobs diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala new file mode 100644 index 0000000000000..94beb590f52d6 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.scheduler + +import org.apache.spark.streaming.receiver.ReceivedBlockStoreResult + +/** Information about blocks received by the receiver */ +private[streaming] case class ReceivedBlockInfo( + streamId: Int, + numRecords: Long, + blockStoreResult: ReceivedBlockStoreResult + ) + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 7149dbc12a365..d696563bcee83 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -21,21 +21,12 @@ import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} import scala.language.existentials import akka.actor._ -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{SerializableWritable, Logging, SparkEnv, SparkException} import org.apache.spark.SparkContext._ -import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver.{Receiver, ReceiverSupervisorImpl, StopReceiver} import org.apache.spark.util.AkkaUtils -/** Information about blocks received by the receiver */ -private[streaming] case class ReceivedBlockInfo( - streamId: Int, - blockId: StreamBlockId, - numRecords: Long, - metadata: Any - ) - /** * Messages used by the NetworkReceiver and the ReceiverTracker to communicate * with each other. @@ -153,7 +144,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { def addBlocks(receivedBlockInfo: ReceivedBlockInfo) { getReceivedBlockInfoQueue(receivedBlockInfo.streamId) += receivedBlockInfo logDebug("Stream " + receivedBlockInfo.streamId + " received new blocks: " + - receivedBlockInfo.blockId) + receivedBlockInfo.blockStoreResult.blockId) } /** Report error sent by a receiver */ @@ -188,6 +179,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { sender ! true case AddBlock(receivedBlockInfo) => addBlocks(receivedBlockInfo) + sender ! true case ReportError(streamId, message, error) => reportError(streamId, message, error) case DeregisterReceiver(streamId, message, error) => @@ -252,6 +244,9 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { ssc.sc.makeRDD(receivers, receivers.size) } + val checkpointDirOption = Option(ssc.checkpointDir) + val serializableHadoopConf = new SerializableWritable(ssc.sparkContext.hadoopConfiguration) + // Function to start the receiver on the worker node val startReceiver = (iterator: Iterator[Receiver[_]]) => { if (!iterator.hasNext) { @@ -259,9 +254,10 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { "Could not start receiver as object not found.") } val receiver = iterator.next() - val executor = new ReceiverSupervisorImpl(receiver, SparkEnv.get) - executor.start() - executor.awaitTermination() + val supervisor = new ReceiverSupervisorImpl( + receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) + supervisor.start() + supervisor.awaitTermination() } // Run the dummy Spark job to ensure that all slaves have registered. // This avoids all the receivers to be scheduled on the same node. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala index 92bad7a882a65..003989092a42a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala @@ -52,4 +52,3 @@ private[streaming] class WriteAheadLogRandomReader(path: String, conf: Configura HdfsUtils.checkState(!closed, "Stream is closed. Create a new Reader to read from the file.") } } - diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala new file mode 100644 index 0000000000000..ad1a6f01b3a57 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import java.io.File +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.postfixOps + +import akka.actor.{ActorSystem, Props} +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark._ +import org.apache.spark.network.nio.NioBlockTransferService +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.storage._ +import org.apache.spark.streaming.receiver._ +import org.apache.spark.streaming.util._ +import org.apache.spark.util.AkkaUtils +import WriteAheadLogBasedBlockHandler._ +import WriteAheadLogSuite._ + +class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { + + val conf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.rollingInterval", "1") + val hadoopConf = new Configuration() + val storageLevel = StorageLevel.MEMORY_ONLY_SER + val streamId = 1 + val securityMgr = new SecurityManager(conf) + val mapOutputTracker = new MapOutputTrackerMaster(conf) + val shuffleManager = new HashShuffleManager(conf) + val serializer = new KryoSerializer(conf) + val manualClock = new ManualClock + val blockManagerSize = 10000000 + + var actorSystem: ActorSystem = null + var blockManagerMaster: BlockManagerMaster = null + var blockManager: BlockManager = null + var tempDirectory: File = null + + before { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) + this.actorSystem = actorSystem + conf.set("spark.driver.port", boundPort.toString) + + blockManagerMaster = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf, true) + + blockManager = new BlockManager("bm", actorSystem, blockManagerMaster, serializer, + blockManagerSize, conf, mapOutputTracker, shuffleManager, + new NioBlockTransferService(conf, securityMgr)) + + tempDirectory = Files.createTempDir() + manualClock.setTime(0) + } + + after { + if (blockManager != null) { + blockManager.stop() + blockManager = null + } + if (blockManagerMaster != null) { + blockManagerMaster.stop() + blockManagerMaster = null + } + actorSystem.shutdown() + actorSystem.awaitTermination() + actorSystem = null + + if (tempDirectory != null && tempDirectory.exists()) { + FileUtils.deleteDirectory(tempDirectory) + tempDirectory = null + } + } + + test("BlockManagerBasedBlockHandler - store blocks") { + withBlockManagerBasedBlockHandler { handler => + testBlockStoring(handler) { case (data, blockIds, storeResults) => + // Verify the data in block manager is correct + val storedData = blockIds.flatMap { blockId => + blockManager.getLocal(blockId).map { _.data.map {_.toString}.toList }.getOrElse(List.empty) + }.toList + storedData shouldEqual data + + // Verify that the store results are instances of BlockManagerBasedStoreResult + assert( + storeResults.forall { _.isInstanceOf[BlockManagerBasedStoreResult] }, + "Unexpected store result type" + ) + } + } + } + + test("BlockManagerBasedBlockHandler - handle errors in storing block") { + withBlockManagerBasedBlockHandler { handler => + testErrorHandling(handler) + } + } + + test("WriteAheadLogBasedBlockHandler - store blocks") { + withWriteAheadLogBasedBlockHandler { handler => + testBlockStoring(handler) { case (data, blockIds, storeResults) => + // Verify the data in block manager is correct + val storedData = blockIds.flatMap { blockId => + blockManager.getLocal(blockId).map { _.data.map {_.toString}.toList }.getOrElse(List.empty) + }.toList + storedData shouldEqual data + + // Verify that the store results are instances of WriteAheadLogBasedStoreResult + assert( + storeResults.forall { _.isInstanceOf[WriteAheadLogBasedStoreResult] }, + "Unexpected store result type" + ) + // Verify the data in write ahead log files is correct + val fileSegments = storeResults.map { _.asInstanceOf[WriteAheadLogBasedStoreResult].segment} + val loggedData = fileSegments.flatMap { segment => + val reader = new WriteAheadLogRandomReader(segment.path, hadoopConf) + val bytes = reader.read(segment) + reader.close() + blockManager.dataDeserialize(generateBlockId(), bytes).toList + } + loggedData shouldEqual data + } + } + } + + test("WriteAheadLogBasedBlockHandler - handle errors in storing block") { + withWriteAheadLogBasedBlockHandler { handler => + testErrorHandling(handler) + } + } + + test("WriteAheadLogBasedBlockHandler - cleanup old blocks") { + withWriteAheadLogBasedBlockHandler { handler => + val blocks = Seq.tabulate(10) { i => IteratorBlock(Iterator(1 to i)) } + storeBlocks(handler, blocks) + + val preCleanupLogFiles = getWriteAheadLogFiles() + preCleanupLogFiles.size should be > 1 + + // this depends on the number of blocks inserted using generateAndStoreData() + manualClock.currentTime() shouldEqual 5000L + + val cleanupThreshTime = 3000L + handler.cleanupOldBlock(cleanupThreshTime) + eventually(timeout(10000 millis), interval(10 millis)) { + getWriteAheadLogFiles().size should be < preCleanupLogFiles.size + } + } + } + + /** + * Test storing of data using different forms of ReceivedBlocks and verify that they succeeded + * using the given verification function + */ + private def testBlockStoring(receivedBlockHandler: ReceivedBlockHandler) + (verifyFunc: (Seq[String], Seq[StreamBlockId], Seq[ReceivedBlockStoreResult]) => Unit) { + val data = Seq.tabulate(100) { _.toString } + + def storeAndVerify(blocks: Seq[ReceivedBlock]) { + blocks should not be empty + val (blockIds, storeResults) = storeBlocks(receivedBlockHandler, blocks) + withClue(s"Testing with ${blocks.head.getClass.getSimpleName}s:") { + // Verify returns store results have correct block ids + (storeResults.map { _.blockId }) shouldEqual blockIds + + // Call handler-specific verification function + verifyFunc(data, blockIds, storeResults) + } + } + + def dataToByteBuffer(b: Seq[String]) = blockManager.dataSerialize(generateBlockId, b.iterator) + + val blocks = data.grouped(10).toSeq + + storeAndVerify(blocks.map { b => IteratorBlock(b.toIterator) }) + storeAndVerify(blocks.map { b => ArrayBufferBlock(new ArrayBuffer ++= b) }) + storeAndVerify(blocks.map { b => ByteBufferBlock(dataToByteBuffer(b)) }) + } + + /** 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)) + } + + /** Instantiate a WriteAheadLogBasedBlockHandler and run a code with it */ + private def withWriteAheadLogBasedBlockHandler(body: WriteAheadLogBasedBlockHandler => Unit) { + val receivedBlockHandler = new WriteAheadLogBasedBlockHandler(blockManager, 1, + storageLevel, conf, hadoopConf, tempDirectory.toString, manualClock) + try { + body(receivedBlockHandler) + } finally { + receivedBlockHandler.stop() + } + } + + /** Store blocks using a handler */ + private def storeBlocks( + receivedBlockHandler: ReceivedBlockHandler, + blocks: Seq[ReceivedBlock] + ): (Seq[StreamBlockId], Seq[ReceivedBlockStoreResult]) = { + val blockIds = Seq.fill(blocks.size)(generateBlockId()) + val storeResults = blocks.zip(blockIds).map { + case (block, id) => + manualClock.addToTime(500) // log rolling interval set to 1000 ms through SparkConf + logDebug("Inserting block " + id) + receivedBlockHandler.storeBlock(id, block) + }.toList + logDebug("Done inserting") + (blockIds, storeResults) + } + + private def getWriteAheadLogFiles(): Seq[String] = { + getLogFilesInDirectory(checkpointDirToLogDir(tempDirectory.toString, streamId)) + } + + private def generateBlockId(): StreamBlockId = StreamBlockId(streamId, scala.util.Random.nextLong) +} 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 5eba93c208c50..1956a4f1db90a 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 @@ -58,7 +58,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { test("WriteAheadLogWriter - writing data") { val dataToWrite = generateRandomData() val segments = writeDataUsingWriter(testFile, dataToWrite) - val writtenData = readDataManually(testFile, segments) + val writtenData = readDataManually(segments) assert(writtenData === dataToWrite) } @@ -67,7 +67,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { val writer = new WriteAheadLogWriter(testFile, hadoopConf) dataToWrite.foreach { data => val segment = writer.write(stringToByteBuffer(data)) - val dataRead = readDataManually(testFile, Seq(segment)).head + val dataRead = readDataManually(Seq(segment)).head assert(data === dataRead) } writer.close() @@ -281,14 +281,20 @@ object WriteAheadLogSuite { } /** Read data from a segments of a log file directly and return the list of byte buffers.*/ - def readDataManually(file: String, segments: Seq[WriteAheadLogFileSegment]): Seq[String] = { - val reader = HdfsUtils.getInputStream(file, hadoopConf) - segments.map { x => - reader.seek(x.offset) - val data = new Array[Byte](x.length) - reader.readInt() - reader.readFully(data) - Utils.deserialize[String](data) + def readDataManually(segments: Seq[WriteAheadLogFileSegment]): Seq[String] = { + segments.map { segment => + val reader = HdfsUtils.getInputStream(segment.path, hadoopConf) + try { + reader.seek(segment.offset) + val bytes = new Array[Byte](segment.length) + reader.readInt() + reader.readFully(bytes) + val data = Utils.deserialize[String](bytes) + reader.close() + data + } finally { + reader.close() + } } } @@ -335,9 +341,11 @@ object WriteAheadLogSuite { val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { - fileSystem.listStatus(logDirectoryPath).map { - _.getPath.toString.stripPrefix("file:") - }.sorted + fileSystem.listStatus(logDirectoryPath).map { _.getPath() }.sortBy { + _.getName().split("-")(1).toLong + }.map { + _.toString.stripPrefix("file:") + } } else { Seq.empty } From fb1fbca204250840ffdbc0fcbf80b8dfeebf9edb Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 30 Oct 2014 15:17:02 -0700 Subject: [PATCH 087/115] [SPARK-4027][Streaming] WriteAheadLogBackedBlockRDD to read received either from BlockManager or WAL in HDFS As part of the initiative of preventing data loss on streaming driver failure, this sub-task implements a BlockRDD that is backed by HDFS. This BlockRDD can either read data from the Spark's BlockManager, or read the data from file-segments in write ahead log in HDFS. Most of this code has been written by @harishreedharan Author: Tathagata Das Author: Hari Shreedharan Closes #2931 from tdas/driver-ha-rdd and squashes the following commits: 209e49c [Tathagata Das] Better fix to style issue. 4a5866f [Tathagata Das] Addressed one more comment. ed5fbf0 [Tathagata Das] Minor updates. b0a18b1 [Tathagata Das] Fixed import order. 20aa7c6 [Tathagata Das] Fixed more line length issues. 29aa099 [Tathagata Das] Fixed line length issues. 9e47b5b [Tathagata Das] Renamed class, simplified+added unit tests. 6e1bfb8 [Tathagata Das] Tweaks testuite to create spark contxt lazily to prevent contxt leaks. 9c86a61 [Tathagata Das] Merge pull request #22 from harishreedharan/driver-ha-rdd 2878c38 [Hari Shreedharan] Shutdown spark context after tests. Formatting/minor fixes c709f2f [Tathagata Das] Merge pull request #21 from harishreedharan/driver-ha-rdd 5cce16f [Hari Shreedharan] Make sure getBlockLocations uses offset and length to find the blocks on HDFS eadde56 [Tathagata Das] Transferred HDFSBackedBlockRDD for the driver-ha-working branch --- .../scala/org/apache/spark/rdd/BlockRDD.scala | 4 + .../rdd/WriteAheadLogBackedBlockRDD.scala | 125 +++++++++++++++ .../spark/streaming/util/HdfsUtils.scala | 8 +- .../WriteAheadLogBackedBlockRDDSuite.scala | 151 ++++++++++++++++++ 4 files changed, 285 insertions(+), 3 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 2673ec22509e9..fffa1911f5bc2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -84,5 +84,9 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds "Attempted to use %s after its blocks have been removed!".format(toString)) } } + + protected def getBlockIdLocations(): Map[BlockId, Seq[String]] = { + locations_ + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala new file mode 100644 index 0000000000000..23295bf658712 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.rdd + +import scala.reflect.ClassTag + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark._ +import org.apache.spark.rdd.BlockRDD +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader} + +/** + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]]. + * It contains information about the id of the blocks having this partition's data and + * the segment of the write ahead log that backs the partition. + * @param index index of the partition + * @param blockId id of the block having the partition data + * @param segment segment of the write ahead log having the partition data + */ +private[streaming] +class WriteAheadLogBackedBlockRDDPartition( + val index: Int, + val blockId: BlockId, + val segment: WriteAheadLogFileSegment) + extends Partition + + +/** + * This class represents a special case of the BlockRDD where the data blocks in + * the block manager are also backed by segments in write ahead logs. For reading + * the data, this RDD first looks up the blocks by their ids in the block manager. + * If it does not find them, it looks up the corresponding file segment. + * + * @param sc SparkContext + * @param hadoopConfig Hadoop configuration + * @param blockIds Ids of the blocks that contains this RDD's data + * @param segments Segments in write ahead logs that contain this RDD's data + * @param storeInBlockManager Whether to store in the block manager after reading from the segment + * @param storageLevel storage level to store when storing in block manager + * (applicable when storeInBlockManager = true) + */ +private[streaming] +class WriteAheadLogBackedBlockRDD[T: ClassTag]( + @transient sc: SparkContext, + @transient hadoopConfig: Configuration, + @transient blockIds: Array[BlockId], + @transient segments: Array[WriteAheadLogFileSegment], + storeInBlockManager: Boolean, + storageLevel: StorageLevel) + extends BlockRDD[T](sc, blockIds) { + + require( + blockIds.length == segments.length, + s"Number of block ids (${blockIds.length}) must be " + + s"the same as number of segments (${segments.length}})!") + + // Hadoop configuration is not serializable, so broadcast it as a serializable. + private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig) + + override def getPartitions: Array[Partition] = { + assertValid() + Array.tabulate(blockIds.size) { i => + new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i)) + } + } + + /** + * Gets the partition data by getting the corresponding block from the block manager. + * If the block does not exist, then the data is read from the corresponding segment + * in write ahead log files. + */ + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + assertValid() + val hadoopConf = broadcastedHadoopConf.value + val blockManager = SparkEnv.get.blockManager + val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] + val blockId = partition.blockId + blockManager.get(blockId) match { + case Some(block) => // Data is in Block Manager + val iterator = block.data.asInstanceOf[Iterator[T]] + logDebug(s"Read partition data of $this from block manager, block $blockId") + iterator + case None => // Data not found in Block Manager, grab it from write ahead log file + val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf) + val dataRead = reader.read(partition.segment) + reader.close() + logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}") + if (storeInBlockManager) { + blockManager.putBytes(blockId, dataRead, storageLevel) + logDebug(s"Stored partition data of $this into block manager with level $storageLevel") + dataRead.rewind() + } + blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]] + } + } + + /** + * Get the preferred location of the partition. This returns the locations of the block + * if it is present in the block manager, else it returns the location of the + * corresponding segment in HDFS. + */ + override def getPreferredLocations(split: Partition): Seq[String] = { + val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] + val blockLocations = getBlockIdLocations().get(partition.blockId) + def segmentLocations = HdfsUtils.getFileSegmentLocations( + partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig) + blockLocations.getOrElse(segmentLocations) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index 491f1175576e6..27a28bab83ed5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -52,12 +52,14 @@ private[streaming] object HdfsUtils { } } - def getBlockLocations(path: String, conf: Configuration): Option[Array[String]] = { + /** Get the locations of the HDFS blocks containing the given file segment. */ + def getFileSegmentLocations( + path: String, offset: Long, length: Long, conf: Configuration): Array[String] = { val dfsPath = new Path(path) val dfs = getFileSystemForPath(dfsPath, conf) val fileStatus = dfs.getFileStatus(dfsPath) - val blockLocs = Option(dfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen)) - blockLocs.map(_.flatMap(_.getHosts)) + val blockLocs = Option(dfs.getFileBlockLocations(fileStatus, offset, length)) + blockLocs.map(_.flatMap(_.getHosts)).getOrElse(Array.empty) } def getFileSystemForPath(path: Path, conf: Configuration): FileSystem = { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala new file mode 100644 index 0000000000000..10160244bcc91 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.rdd + +import java.io.File + +import scala.util.Random + +import com.google.common.io.Files +import org.apache.hadoop.conf.Configuration +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter} + +class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { + val conf = new SparkConf() + .setMaster("local[2]") + .setAppName(this.getClass.getSimpleName) + val hadoopConf = new Configuration() + + var sparkContext: SparkContext = null + var blockManager: BlockManager = null + var dir: File = null + + override def beforeAll(): Unit = { + sparkContext = new SparkContext(conf) + blockManager = sparkContext.env.blockManager + dir = Files.createTempDir() + } + + override def afterAll(): Unit = { + // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests. + sparkContext.stop() + dir.delete() + System.clearProperty("spark.driver.port") + } + + test("Read data available in block manager and write ahead log") { + testRDD(5, 5) + } + + test("Read data available only in block manager, not in write ahead log") { + testRDD(5, 0) + } + + test("Read data available only in write ahead log, not in block manager") { + testRDD(0, 5) + } + + test("Read data available only in write ahead log, and test storing in block manager") { + testRDD(0, 5, testStoreInBM = true) + } + + test("Read data with partially available in block manager, and rest in write ahead log") { + testRDD(3, 2) + } + + /** + * Test the WriteAheadLogBackedRDD, by writing some partitions of the data to block manager + * and the rest to a write ahead log, and then reading reading it all back using the RDD. + * It can also test if the partitions that were read from the log were again stored in + * block manager. + * @param numPartitionsInBM Number of partitions to write to the Block Manager + * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log + * @param testStoreInBM Test whether blocks read from log are stored back into block manager + */ + private def testRDD(numPartitionsInBM: Int, numPartitionsInWAL: Int, testStoreInBM: Boolean = false) { + val numBlocks = numPartitionsInBM + numPartitionsInWAL + val data = Seq.fill(numBlocks, 10)(scala.util.Random.nextString(50)) + + // Put the necessary blocks in the block manager + val blockIds = Array.fill(numBlocks)(StreamBlockId(Random.nextInt(), Random.nextInt())) + data.zip(blockIds).take(numPartitionsInBM).foreach { case(block, blockId) => + blockManager.putIterator(blockId, block.iterator, StorageLevel.MEMORY_ONLY_SER) + } + + // Generate write ahead log segments + val segments = generateFakeSegments(numPartitionsInBM) ++ + writeLogSegments(data.takeRight(numPartitionsInWAL), blockIds.takeRight(numPartitionsInWAL)) + + // Make sure that the left `numPartitionsInBM` blocks are in block manager, and others are not + require( + blockIds.take(numPartitionsInBM).forall(blockManager.get(_).nonEmpty), + "Expected blocks not in BlockManager" + ) + require( + blockIds.takeRight(numPartitionsInWAL).forall(blockManager.get(_).isEmpty), + "Unexpected blocks in BlockManager" + ) + + // Make sure that the right `numPartitionsInWAL` blocks are in write ahead logs, and other are not + require( + segments.takeRight(numPartitionsInWAL).forall(s => + new File(s.path.stripPrefix("file://")).exists()), + "Expected blocks not in write ahead log" + ) + require( + segments.take(numPartitionsInBM).forall(s => + !new File(s.path.stripPrefix("file://")).exists()), + "Unexpected blocks in write ahead log" + ) + + // Create the RDD and verify whether the returned data is correct + val rdd = new WriteAheadLogBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, + segments.toArray, storeInBlockManager = false, StorageLevel.MEMORY_ONLY) + assert(rdd.collect() === data.flatten) + + if (testStoreInBM) { + val rdd2 = new WriteAheadLogBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, + segments.toArray, storeInBlockManager = true, StorageLevel.MEMORY_ONLY) + assert(rdd2.collect() === data.flatten) + assert( + blockIds.forall(blockManager.get(_).nonEmpty), + "All blocks not found in block manager" + ) + } + } + + private def writeLogSegments( + blockData: Seq[Seq[String]], + blockIds: Seq[BlockId] + ): Seq[WriteAheadLogFileSegment] = { + require(blockData.size === blockIds.size) + val writer = new WriteAheadLogWriter(new File(dir, Random.nextString(10)).toString, hadoopConf) + val segments = blockData.zip(blockIds).map { case (data, id) => + writer.write(blockManager.dataSerialize(id, data.iterator)) + } + writer.close() + segments + } + + private def generateFakeSegments(count: Int): Seq[WriteAheadLogFileSegment] = { + Array.fill(count)(new WriteAheadLogFileSegment("random", 0l, 0)) + } +} From 9142c9b80bfe12e0be8a2b795bf52e403b2c5f30 Mon Sep 17 00:00:00 2001 From: Grace Date: Thu, 30 Oct 2014 15:27:32 -0700 Subject: [PATCH 088/115] [SPARK-4078] New FsPermission instance w/o FsPermission.createImmutable in eventlog By default, Spark builds its package against Hadoop 1.0.4 version. In that version, it has some FsPermission bug (see [HADOOP-7629] (https://issues.apache.org/jira/browse/HADOOP-7629) by Todd Lipcon). This bug got fixed since 1.1 version. By using that FsPermission.createImmutable() API, end-user may see some RPC exception like below (if turn on eventlog over HDFS). Here proposes a quick fix to avoid certain exception for all hadoop versions. ``` Exception in thread "main" java.io.IOException: Call to sr484/10.1.2.84:54310 failed on local exception: java.io.EOFException at org.apache.hadoop.ipc.Client.wrapException(Client.java:1150) at org.apache.hadoop.ipc.Client.call(Client.java:1118) at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:229) at $Proxy6.setPermission(Unknown Source) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:85) at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:62) at $Proxy6.setPermission(Unknown Source) at org.apache.hadoop.hdfs.DFSClient.setPermission(DFSClient.java:1285) at org.apache.hadoop.hdfs.DistributedFileSystem.setPermission(DistributedFileSystem.java:572) at org.apache.spark.util.FileLogger.createLogDir(FileLogger.scala:138) at org.apache.spark.util.FileLogger.start(FileLogger.scala:115) at org.apache.spark.scheduler.EventLoggingListener.start(EventLoggingListener.scala:74) at org.apache.spark.SparkContext.(SparkContext.scala:324) ``` Author: Grace Closes #2892 from GraceH/eventlog-rpc and squashes the following commits: 58ea038 [Grace] new FsPermission Instance w/o FsPermission.createImmutable --- .../scala/org/apache/spark/scheduler/EventLoggingListener.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 100c9ba9b7809..597dbc884913c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -142,7 +142,7 @@ private[spark] object EventLoggingListener extends Logging { val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" - val LOG_FILE_PERMISSIONS = FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) + val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) // A cache for compression codecs to avoid creating the same codec many times private val codecMap = new mutable.HashMap[String, CompressionCodec] From 24c5129257ce6e3b734f168e860b714c2730b55f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:29:07 -0700 Subject: [PATCH 089/115] [SPARK-3319] [SPARK-3338] Resolve Spark submit config paths The bulk of this PR is comprised of tests. All changes in functionality are made in `SparkSubmit.scala` (~20 lines). **SPARK-3319.** There is currently a divergence in behavior when the user passes in additional jars through `--jars` and through setting `spark.jars` in the default properties file. The former will happily resolve the paths (e.g. convert `my.jar` to `file:/absolute/path/to/my.jar`), while the latter does not. We should resolve paths consistently in both cases. This also applies to the following pairs of command line arguments and Spark configs: - `--jars` ~ `spark.jars` - `--files` ~ `spark.files` / `spark.yarn.dist.files` - `--archives` ~ `spark.yarn.dist.archives` - `--py-files` ~ `spark.submit.pyFiles` **SPARK-3338.** This PR also fixes the following bug: if the user sets `spark.submit.pyFiles` in his/her properties file, it does not actually get picked up even if `--py-files` is not set. This is simply because the config is overridden by an empty string. Author: Andrew Or Author: Andrew Or Closes #2232 from andrewor14/resolve-config-paths and squashes the following commits: fff2869 [Andrew Or] Add spark.yarn.jar da3a1c1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into resolve-config-paths f0fae64 [Andrew Or] Merge branch 'master' of github.com:apache/spark into resolve-config-paths 05e03d6 [Andrew Or] Add tests for resolving both command line and config paths 460117e [Andrew Or] Resolve config paths properly fe039d3 [Andrew Or] Beef up tests to test fixed-pointed-ness of Utils.resolveURI(s) --- .../org/apache/spark/deploy/SparkSubmit.scala | 28 ++++- .../spark/deploy/SparkSubmitSuite.scala | 106 +++++++++++++++++- .../org/apache/spark/util/UtilsSuite.scala | 38 +++++-- 3 files changed, 158 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index f97bf67fa5a3b..0379adeb07b83 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -158,8 +158,9 @@ object SparkSubmit { args.files = mergeFileLists(args.files, args.primaryResource) } args.files = mergeFileLists(args.files, args.pyFiles) - // Format python file paths properly before adding them to the PYTHONPATH - sysProps("spark.submit.pyFiles") = PythonRunner.formatPaths(args.pyFiles).mkString(",") + if (args.pyFiles != null) { + sysProps("spark.submit.pyFiles") = args.pyFiles + } } // Special flag to avoid deprecation warnings at the client @@ -284,6 +285,29 @@ object SparkSubmit { sysProps.getOrElseUpdate(k, v) } + // Resolve paths in certain spark properties + val pathConfigs = Seq( + "spark.jars", + "spark.files", + "spark.yarn.jar", + "spark.yarn.dist.files", + "spark.yarn.dist.archives") + pathConfigs.foreach { config => + // Replace old URIs with resolved URIs, if they exist + sysProps.get(config).foreach { oldValue => + sysProps(config) = Utils.resolveURIs(oldValue) + } + } + + // Resolve and format python file paths properly before adding them to the PYTHONPATH. + // The resolving part is redundant in the case of --py-files, but necessary if the user + // explicitly sets `spark.submit.pyFiles` in his/her default properties file. + sysProps.get("spark.submit.pyFiles").foreach { pyFiles => + val resolvedPyFiles = Utils.resolveURIs(pyFiles) + val formattedPyFiles = PythonRunner.formatPaths(resolvedPyFiles).mkString(",") + sysProps("spark.submit.pyFiles") = formattedPyFiles + } + (childArgs, childClasspath, sysProps, childMainClass) } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 1cdf50d5c08c7..d8cd0ff2c9026 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -292,7 +292,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { runSparkSubmit(args) } - test("spark submit includes jars passed in through --jar") { + test("includes jars passed in through --jars") { val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) @@ -306,6 +306,110 @@ class SparkSubmitSuite extends FunSuite with Matchers { runSparkSubmit(args) } + test("resolves command line argument paths correctly") { + val jars = "/jar1,/jar2" // --jars + val files = "hdfs:/file1,file2" // --files + val archives = "file:/archive1,archive2" // --archives + val pyFiles = "py-file1,py-file2" // --py-files + + // Test jars and files + val clArgs = Seq( + "--master", "local", + "--class", "org.SomeClass", + "--jars", jars, + "--files", files, + "thejar.jar") + val appArgs = new SparkSubmitArguments(clArgs) + val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3 + appArgs.jars should be (Utils.resolveURIs(jars)) + appArgs.files should be (Utils.resolveURIs(files)) + sysProps("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar")) + sysProps("spark.files") should be (Utils.resolveURIs(files)) + + // Test files and archives (Yarn) + val clArgs2 = Seq( + "--master", "yarn-client", + "--class", "org.SomeClass", + "--files", files, + "--archives", archives, + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3 + appArgs2.files should be (Utils.resolveURIs(files)) + appArgs2.archives should be (Utils.resolveURIs(archives)) + sysProps2("spark.yarn.dist.files") should be (Utils.resolveURIs(files)) + sysProps2("spark.yarn.dist.archives") should be (Utils.resolveURIs(archives)) + + // Test python files + val clArgs3 = Seq( + "--master", "local", + "--py-files", pyFiles, + "mister.py" + ) + val appArgs3 = new SparkSubmitArguments(clArgs3) + val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3 + appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles)) + sysProps3("spark.submit.pyFiles") should be ( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + } + + test("resolves config paths correctly") { + val jars = "/jar1,/jar2" // spark.jars + val files = "hdfs:/file1,file2" // spark.files / spark.yarn.dist.files + val archives = "file:/archive1,archive2" // spark.yarn.dist.archives + val pyFiles = "py-file1,py-file2" // spark.submit.pyFiles + + // Test jars and files + val f1 = File.createTempFile("test-submit-jars-files", "") + val writer1 = new PrintWriter(f1) + writer1.println("spark.jars " + jars) + writer1.println("spark.files " + files) + writer1.close() + val clArgs = Seq( + "--master", "local", + "--class", "org.SomeClass", + "--properties-file", f1.getPath, + "thejar.jar" + ) + val appArgs = new SparkSubmitArguments(clArgs) + val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3 + sysProps("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) + sysProps("spark.files") should be(Utils.resolveURIs(files)) + + // Test files and archives (Yarn) + val f2 = File.createTempFile("test-submit-files-archives", "") + val writer2 = new PrintWriter(f2) + writer2.println("spark.yarn.dist.files " + files) + writer2.println("spark.yarn.dist.archives " + archives) + writer2.close() + val clArgs2 = Seq( + "--master", "yarn-client", + "--class", "org.SomeClass", + "--properties-file", f2.getPath, + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3 + sysProps2("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) + sysProps2("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) + + // Test python files + val f3 = File.createTempFile("test-submit-python-files", "") + val writer3 = new PrintWriter(f3) + writer3.println("spark.submit.pyFiles " + pyFiles) + writer3.close() + val clArgs3 = Seq( + "--master", "local", + "--properties-file", f3.getPath, + "mister.py" + ) + val appArgs3 = new SparkSubmitArguments(clArgs3) + val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3 + sysProps3("spark.submit.pyFiles") should be( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + } + test("SPARK_CONF_DIR overrides spark-defaults.conf") { forConfDir(Map("spark.executor.memory" -> "2.3g")) { path => val unusedJar = TestUtils.createJarWithClasses(Seq.empty) diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 1c112334cc2f5..8ffe3e2b139c3 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -217,9 +217,14 @@ class UtilsSuite extends FunSuite { test("resolveURI") { def assertResolves(before: String, after: String, testWindows: Boolean = false): Unit = { - assume(before.split(",").length == 1) - assert(Utils.resolveURI(before, testWindows) === new URI(after)) - assert(Utils.resolveURI(after, testWindows) === new URI(after)) + // This should test only single paths + assume(before.split(",").length === 1) + // Repeated invocations of resolveURI should yield the same result + def resolve(uri: String): String = Utils.resolveURI(uri, testWindows).toString + assert(resolve(after) === after) + assert(resolve(resolve(after)) === after) + assert(resolve(resolve(resolve(after))) === after) + // Also test resolveURIs with single paths assert(new URI(Utils.resolveURIs(before, testWindows)) === new URI(after)) assert(new URI(Utils.resolveURIs(after, testWindows)) === new URI(after)) } @@ -235,16 +240,27 @@ class UtilsSuite extends FunSuite { assertResolves("file:/C:/file.txt#alias.txt", "file:/C:/file.txt#alias.txt", testWindows = true) intercept[IllegalArgumentException] { Utils.resolveURI("file:foo") } intercept[IllegalArgumentException] { Utils.resolveURI("file:foo:baby") } + } - // Test resolving comma-delimited paths - assert(Utils.resolveURIs("jar1,jar2") === s"file:$cwd/jar1,file:$cwd/jar2") - assert(Utils.resolveURIs("file:/jar1,file:/jar2") === "file:/jar1,file:/jar2") - assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3") === - s"hdfs:/jar1,file:/jar2,file:$cwd/jar3") - assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3,jar4#jar5") === + test("resolveURIs with multiple paths") { + def assertResolves(before: String, after: String, testWindows: Boolean = false): Unit = { + assume(before.split(",").length > 1) + assert(Utils.resolveURIs(before, testWindows) === after) + assert(Utils.resolveURIs(after, testWindows) === after) + // Repeated invocations of resolveURIs should yield the same result + def resolve(uri: String): String = Utils.resolveURIs(uri, testWindows) + assert(resolve(after) === after) + assert(resolve(resolve(after)) === after) + assert(resolve(resolve(resolve(after))) === after) + } + val cwd = System.getProperty("user.dir") + assertResolves("jar1,jar2", s"file:$cwd/jar1,file:$cwd/jar2") + assertResolves("file:/jar1,file:/jar2", "file:/jar1,file:/jar2") + assertResolves("hdfs:/jar1,file:/jar2,jar3", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3") + assertResolves("hdfs:/jar1,file:/jar2,jar3,jar4#jar5", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:$cwd/jar4#jar5") - assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi", testWindows = true) === - s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi") + assertResolves("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi", + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi", testWindows = true) } test("nonLocalPaths") { From 26f092d4e32cc1f7e279646075eaf1e495395923 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:31:23 -0700 Subject: [PATCH 090/115] [SPARK-4138][SPARK-4139] Improve dynamic allocation settings This should be merged after #2746 (SPARK-3795). **SPARK-4138**. If the user sets both the number of executors and `spark.dynamicAllocation.enabled`, we should throw an exception. **SPARK-4139**. If the user sets `spark.dynamicAllocation.enabled`, we should use the max number of executors as the starting number of executors because the first job is likely to run immediately after application startup. If the latter is not set, throw an exception. Author: Andrew Or Closes #3002 from andrewor14/yarn-set-executors and squashes the following commits: c528fce [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-set-executors 55d4699 [Andrew Or] Bug fix: `isDynamicAllocationEnabled` was always false 2b0ccec [Andrew Or] Start the number of executors at the max 022bfde [Andrew Or] Guard against incompatible settings of number of executors --- .../yarn/ApplicationMasterArguments.scala | 3 +- .../spark/deploy/yarn/ClientArguments.scala | 30 ++++++++++++++----- .../deploy/yarn/YarnSparkHadoopUtil.scala | 2 ++ .../cluster/YarnClusterSchedulerBackend.scala | 4 +-- 4 files changed, 29 insertions(+), 10 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 5c54e3400301a..104db4f65f030 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import org.apache.spark.util.{MemoryParam, IntParam} +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import collection.mutable.ArrayBuffer class ApplicationMasterArguments(val args: Array[String]) { @@ -26,7 +27,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 var executorCores = 1 - var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS + var numExecutors = DEFAULT_NUMBER_EXECUTORS parseArgs(args.toList) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index a12f82d2fbe70..4d859450efc63 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -20,8 +20,8 @@ package org.apache.spark.deploy.yarn import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf -import org.apache.spark.util.{Utils, IntParam, MemoryParam} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ +import org.apache.spark.util.{Utils, IntParam, MemoryParam} // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) { @@ -33,23 +33,25 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 // MB var executorCores = 1 - var numExecutors = 2 + var numExecutors = DEFAULT_NUMBER_EXECUTORS var amQueue = sparkConf.get("spark.yarn.queue", "default") var amMemory: Int = 512 // MB var appName: String = "Spark" var priority = 0 - parseArgs(args.toList) - loadEnvironmentArgs() - // Additional memory to allocate to containers // For now, use driver's memory overhead as our AM container's memory overhead - val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", + val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toInt, MEMORY_OVERHEAD_MIN)) - val executorMemoryOverhead = sparkConf.getInt("spark.yarn.executor.memoryOverhead", + val executorMemoryOverhead = sparkConf.getInt("spark.yarn.executor.memoryOverhead", math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)) + private val isDynamicAllocationEnabled = + sparkConf.getBoolean("spark.dynamicAllocation.enabled", false) + + parseArgs(args.toList) + loadEnvironmentArgs() validateArgs() /** Load any default arguments provided through environment variables and Spark properties. */ @@ -64,6 +66,15 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) .orNull + // If dynamic allocation is enabled, start at the max number of executors + if (isDynamicAllocationEnabled) { + val maxExecutorsConf = "spark.dynamicAllocation.maxExecutors" + if (!sparkConf.contains(maxExecutorsConf)) { + throw new IllegalArgumentException( + s"$maxExecutorsConf must be set if dynamic allocation is enabled!") + } + numExecutors = sparkConf.get(maxExecutorsConf).toInt + } } /** @@ -113,6 +124,11 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) if (args(0) == "--num-workers") { println("--num-workers is deprecated. Use --num-executors instead.") } + // Dynamic allocation is not compatible with this option + if (isDynamicAllocationEnabled) { + throw new IllegalArgumentException("Explicitly setting the number " + + "of executors is not compatible with spark.dynamicAllocation.enabled!") + } numExecutors = value args = tail diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index e1e0144f46fe9..7d453ecb7983c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -93,6 +93,8 @@ object YarnSparkHadoopUtil { val ANY_HOST = "*" + val DEFAULT_NUMBER_EXECUTORS = 2 + // All RM requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) val RM_REQUEST_PRIORITY = 1 diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index a96a54f66824c..b1de81e6a8b0f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.SparkContext -import org.apache.spark.deploy.yarn.ApplicationMasterArguments +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.IntParam @@ -29,7 +29,7 @@ private[spark] class YarnClusterSchedulerBackend( override def start() { super.start() - totalExpectedExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS + totalExpectedExecutors = DEFAULT_NUMBER_EXECUTORS if (System.getenv("SPARK_EXECUTOR_INSTANCES") != null) { totalExpectedExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES")) .getOrElse(totalExpectedExecutors) From 5231a3f228b5482cba09ae23a9f68498eba03c88 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:32:11 -0700 Subject: [PATCH 091/115] [Minor] A few typos in comments and log messages Author: Andrew Or Author: Andrew Or Closes #3021 from andrewor14/typos and squashes the following commits: daaf417 [Andrew Or] Merge branch 'master' of github.com:apache/spark into typos 4838ae4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into typos 026d426 [Andrew Or] Merge branch 'master' of github.com:andrewor14/spark into typos a81ae8f [Andrew Or] Some typos --- .../scala/org/apache/spark/scheduler/TaskResultGetter.scala | 2 +- .../main/scala/org/apache/spark/scheduler/TaskScheduler.scala | 4 ++-- .../apache/spark/deploy/yarn/ApplicationMasterArguments.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 3f345ceeaaf7a..4b5be68ec5f92 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -93,7 +93,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } } catch { case cnd: ClassNotFoundException => - // Log an error but keep going here -- the task failed, so not catastropic if we can't + // Log an error but keep going here -- the task failed, so not catastrophic if we can't // deserialize the reason. val loader = Utils.getContextOrSparkClassLoader logError( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index a129a434c9a1a..f095915352b17 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -23,7 +23,7 @@ import org.apache.spark.storage.BlockManagerId /** * Low-level task scheduler interface, currently implemented exclusively by TaskSchedulerImpl. - * This interface allows plugging in different task schedulers. Each TaskScheduler schedulers tasks + * This interface allows plugging in different task schedulers. Each TaskScheduler schedules tasks * for a single SparkContext. These schedulers get sets of tasks submitted to them from the * DAGScheduler for each stage, and are responsible for sending the tasks to the cluster, running * them, retrying if there are failures, and mitigating stragglers. They return events to the @@ -41,7 +41,7 @@ private[spark] trait TaskScheduler { // Invoked after system has successfully initialized (typically in spark context). // Yarn uses this to bootstrap allocation of resources based on preferred locations, - // wait for slave registerations, etc. + // wait for slave registrations, etc. def postStartHook() { } // Disconnect from the cluster. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 104db4f65f030..8b32c76d14037 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -82,7 +82,7 @@ class ApplicationMasterArguments(val args: Array[String]) { | --jar JAR_PATH Path to your application's JAR file | --class CLASS_NAME Name of your application's main class | --args ARGS Arguments to be passed to your application's main class. - | Mutliple invocations are possible, each will be passed in order. + | Multiple invocations are possible, each will be passed in order. | --num-executors NUM Number of executors to start (Default: 2) | --executor-cores NUM Number of cores for the executors (Default: 1) | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G) From 9334d699671edd8f18370255017ad40c1d0340ee Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:32:46 -0700 Subject: [PATCH 092/115] [SPARK-4155] Consolidate usages of We use "\" everywhere. Let's not do that. Author: Andrew Or Closes #3020 from andrewor14/consolidate-driver and squashes the following commits: c1c2204 [Andrew Or] Just use "" for local executor ID 3d751e9 [Andrew Or] Consolidate usages of --- .../apache/spark/ExecutorAllocationManager.scala | 2 +- .../main/scala/org/apache/spark/SparkContext.scala | 2 ++ core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../apache/spark/scheduler/local/LocalBackend.scala | 4 ++-- .../org/apache/spark/storage/BlockManagerId.scala | 3 ++- .../spark/storage/StorageStatusListener.scala | 13 ++----------- .../org/apache/spark/ui/exec/ExecutorsTab.scala | 6 ++---- .../storage/BlockManagerReplicationSuite.scala | 8 +++++--- .../apache/spark/storage/BlockManagerSuite.scala | 10 ++++++---- 9 files changed, 23 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index b2cf022baf29f..c11f1db0064fd 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -419,7 +419,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { val executorId = blockManagerAdded.blockManagerId.executorId - if (executorId != "") { + if (executorId != SparkContext.DRIVER_IDENTIFIER) { allocationManager.onExecutorAdded(executorId) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 73668e83bbb1d..6bfcd8ceae3c1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1333,6 +1333,8 @@ object SparkContext extends Logging { private[spark] val SPARK_UNKNOWN_USER = "" + private[spark] val DRIVER_IDENTIFIER = "" + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6a6dfda363974..557d2f51283ae 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -156,7 +156,7 @@ object SparkEnv extends Logging { assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") val hostname = conf.get("spark.driver.host") val port = conf.get("spark.driver.port").toInt - create(conf, "", hostname, port, true, isLocal, listenerBus) + create(conf, SparkContext.DRIVER_IDENTIFIER, hostname, port, true, isLocal, listenerBus) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 58b78f041cd85..c0264836de738 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import akka.actor.{Actor, ActorRef, Props} -import org.apache.spark.{Logging, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} @@ -47,7 +47,7 @@ private[spark] class LocalActor( private var freeCores = totalCores - private val localExecutorId = "localhost" + private val localExecutorId = SparkContext.DRIVER_IDENTIFIER private val localExecutorHostname = "localhost" val executor = new Executor( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 259f423c73e6b..b177a59c721df 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap +import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -59,7 +60,7 @@ class BlockManagerId private ( def port: Int = port_ - def isDriver: Boolean = (executorId == "") + def isDriver: Boolean = { executorId == SparkContext.DRIVER_IDENTIFIER } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeUTF(executorId_) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index d9066f766476e..def49e80a3605 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import scala.collection.mutable +import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ @@ -59,10 +60,9 @@ class StorageStatusListener extends SparkListener { val info = taskEnd.taskInfo val metrics = taskEnd.taskMetrics if (info != null && metrics != null) { - val execId = formatExecutorId(info.executorId) val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) if (updatedBlocks.length > 0) { - updateStorageStatus(execId, updatedBlocks) + updateStorageStatus(info.executorId, updatedBlocks) } } } @@ -88,13 +88,4 @@ class StorageStatusListener extends SparkListener { } } - /** - * In the local mode, there is a discrepancy between the executor ID according to the - * task ("localhost") and that according to SparkEnv (""). In the UI, this - * results in duplicate rows for the same executor. Thus, in this mode, we aggregate - * these two rows and use the executor ID of "" to be consistent. - */ - def formatExecutorId(execId: String): String = { - if (execId == "localhost") "" else execId - } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 689cf02b25b70..9e0e71a51a408 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -48,14 +48,14 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp def storageStatusList = storageStatusListener.storageStatusList override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val eid = formatExecutorId(taskStart.taskInfo.executorId) + val eid = taskStart.taskInfo.executorId executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val info = taskEnd.taskInfo if (info != null) { - val eid = formatExecutorId(info.executorId) + val eid = info.executorId executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration taskEnd.reason match { @@ -84,6 +84,4 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp } } - // This addresses executor ID inconsistencies in the local mode - private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 1f1d53a1ee3b0..c6d7105592096 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -27,7 +27,7 @@ import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.network.BlockTransferService import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus @@ -57,7 +57,9 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) - private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + private def makeBlockManager( + maxMem: Long, + name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer) @@ -108,7 +110,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd storeIds.filterNot { _ == stores(2).blockManagerId }) // Add driver store and test whether it is filtered out - val driverStore = makeBlockManager(1000, "") + val driverStore = makeBlockManager(1000, SparkContext.DRIVER_IDENTIFIER) assert(master.getPeers(stores(0).blockManagerId).forall(!_.isDriver)) assert(master.getPeers(stores(1).blockManagerId).forall(!_.isDriver)) assert(master.getPeers(stores(2).blockManagerId).forall(!_.isDriver)) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 9d96202a3e7ac..715b740b857b2 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -37,7 +37,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.executor.DataReadMethod import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus @@ -69,7 +69,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) - private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + private def makeBlockManager( + maxMem: Long, + name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) new BlockManager(name, actorSystem, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer) @@ -790,8 +792,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block store put failure") { // Use Java serializer so we can create an unserializable error. val transfer = new NioBlockTransferService(conf, securityMgr) - store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - mapOutputTracker, shuffleManager, transfer) + store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, actorSystem, master, + new JavaSerializer(conf), 1200, conf, mapOutputTracker, shuffleManager, transfer) // The put should fail since a1 is not serializable. class UnserializableClass From 849b43ec0f9e4f2ef962a054eb78cd0fc94a142a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:33:34 -0700 Subject: [PATCH 093/115] Minor style hot fix after #2711 I had planned to fix this when I merged it but I forgot to. witgo Author: Andrew Or Closes #3018 from andrewor14/command-utils-style and squashes the following commits: c2959fb [Andrew Or] Style hot fix --- .../apache/spark/deploy/worker/CommandUtils.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index aba2e20118d7a..28e9662db5da9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -37,12 +37,12 @@ object CommandUtils extends Logging { * The `env` argument is exposed for testing. */ def buildProcessBuilder( - command: Command, - memory: Int, - sparkHome: String, - substituteArguments: String => String, - classPaths: Seq[String] = Seq[String](), - env: Map[String, String] = sys.env): ProcessBuilder = { + command: Command, + memory: Int, + sparkHome: String, + substituteArguments: String => String, + classPaths: Seq[String] = Seq[String](), + env: Map[String, String] = sys.env): ProcessBuilder = { val localCommand = buildLocalCommand(command, substituteArguments, classPaths, env) val commandSeq = buildCommandSeq(localCommand, memory, sparkHome) val builder = new ProcessBuilder(commandSeq: _*) From d3450578357d6f7598243ee2ab11c338085ad9c1 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 30 Oct 2014 15:33:56 -0700 Subject: [PATCH 094/115] [SPARK-4153][WebUI] Update the sort keys for HistoryPage Sort "Started", "Completed", "Duration" and "Last Updated" by time. Author: zsxwing Closes #3014 from zsxwing/SPARK-4153 and squashes the following commits: ec8b9ad [zsxwing] Sort "Started", "Completed", "Duration" and "Last Updated" by time --- .../org/apache/spark/deploy/history/HistoryPage.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index d25c29113d6da..0e249e51a77d8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -84,11 +84,11 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { {info.id} {info.name} - {startTime} - {endTime} - {duration} + {startTime} + {endTime} + {duration} {info.sparkUser} - {lastUpdated} + {lastUpdated} } } From 2f54543815c0905dc958d444ad638c23a29507c6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:44:29 -0700 Subject: [PATCH 095/115] [SPARK-3661] Respect spark.*.memory in cluster mode This also includes minor re-organization of the code. Tested locally in both client and deploy modes. Author: Andrew Or Author: Andrew Or Closes #2697 from andrewor14/memory-cluster-mode and squashes the following commits: 01d78bc [Andrew Or] Merge branch 'master' of github.com:apache/spark into memory-cluster-mode ccd468b [Andrew Or] Add some comments per Patrick c956577 [Andrew Or] Tweak wording 2b4afa0 [Andrew Or] Unused import 47a5a88 [Andrew Or] Correct Spark properties precedence order bf64717 [Andrew Or] Merge branch 'master' of github.com:apache/spark into memory-cluster-mode dd452d0 [Andrew Or] Respect spark.*.memory in cluster mode --- .../org/apache/spark/deploy/SparkSubmit.scala | 8 +- .../spark/deploy/SparkSubmitArguments.scala | 74 +++++++++++-------- 2 files changed, 45 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 0379adeb07b83..b43e68e40f791 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -274,17 +274,11 @@ object SparkSubmit { } } - // Properties given with --conf are superceded by other options, but take precedence over - // properties in the defaults file. + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sysProps.getOrElseUpdate(k, v) } - // Read from default spark properties, if any - for ((k, v) <- args.defaultSparkProperties) { - sysProps.getOrElseUpdate(k, v) - } - // Resolve paths in certain spark properties val pathConfigs = Seq( "spark.jars", diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 72a452e0aefb5..f0e9ee67f6a67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy import java.util.jar.JarFile -import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.util.Utils @@ -72,39 +71,54 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St defaultProperties } - // Respect SPARK_*_MEMORY for cluster mode - driverMemory = sys.env.get("SPARK_DRIVER_MEMORY").orNull - executorMemory = sys.env.get("SPARK_EXECUTOR_MEMORY").orNull - + // Set parameters from command line arguments parseOpts(args.toList) - mergeSparkProperties() + // Populate `sparkProperties` map from properties file + mergeDefaultSparkProperties() + // Use `sparkProperties` map along with env vars to fill in any missing parameters + loadEnvironmentArguments() + checkRequiredArguments() /** - * Fill in any undefined values based on the default properties file or options passed in through - * the '--conf' flag. + * Merge values from the default properties file with those specified through --conf. + * When this is called, `sparkProperties` is already filled with configs from the latter. */ - private def mergeSparkProperties(): Unit = { + private def mergeDefaultSparkProperties(): Unit = { // Use common defaults file, if not specified by user propertiesFile = Option(propertiesFile).getOrElse(Utils.getDefaultPropertiesFile(env)) + // Honor --conf before the defaults file + defaultSparkProperties.foreach { case (k, v) => + if (!sparkProperties.contains(k)) { + sparkProperties(k) = v + } + } + } - val properties = HashMap[String, String]() - properties.putAll(defaultSparkProperties) - properties.putAll(sparkProperties) - - // Use properties file as fallback for values which have a direct analog to - // arguments in this script. - master = Option(master).orElse(properties.get("spark.master")).orNull - executorMemory = Option(executorMemory).orElse(properties.get("spark.executor.memory")).orNull - executorCores = Option(executorCores).orElse(properties.get("spark.executor.cores")).orNull + /** + * Load arguments from environment variables, Spark properties etc. + */ + private def loadEnvironmentArguments(): Unit = { + master = Option(master) + .orElse(sparkProperties.get("spark.master")) + .orElse(env.get("MASTER")) + .orNull + driverMemory = Option(driverMemory) + .orElse(sparkProperties.get("spark.driver.memory")) + .orElse(env.get("SPARK_DRIVER_MEMORY")) + .orNull + executorMemory = Option(executorMemory) + .orElse(sparkProperties.get("spark.executor.memory")) + .orElse(env.get("SPARK_EXECUTOR_MEMORY")) + .orNull + executorCores = Option(executorCores) + .orElse(sparkProperties.get("spark.executor.cores")) + .orNull totalExecutorCores = Option(totalExecutorCores) - .orElse(properties.get("spark.cores.max")) + .orElse(sparkProperties.get("spark.cores.max")) .orNull - name = Option(name).orElse(properties.get("spark.app.name")).orNull - jars = Option(jars).orElse(properties.get("spark.jars")).orNull - - // This supports env vars in older versions of Spark - master = Option(master).orElse(env.get("MASTER")).orNull + name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull + jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull // Try to set main class from JAR if no --class argument is given @@ -131,7 +145,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } /** Ensure that required fields exists. Call this only once all defaults are loaded. */ - private def checkRequiredArguments() = { + private def checkRequiredArguments(): Unit = { if (args.length == 0) { printUsageAndExit(-1) } @@ -166,7 +180,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } } - override def toString = { + override def toString = { s"""Parsed arguments: | master $master | deployMode $deployMode @@ -174,7 +188,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | executorCores $executorCores | totalExecutorCores $totalExecutorCores | propertiesFile $propertiesFile - | extraSparkProperties $sparkProperties | driverMemory $driverMemory | driverCores $driverCores | driverExtraClassPath $driverExtraClassPath @@ -193,8 +206,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | jars $jars | verbose $verbose | - |Default properties from $propertiesFile: - |${defaultSparkProperties.mkString(" ", "\n ", "\n")} + |Spark properties used, including those specified through + | --conf and those from the properties file $propertiesFile: + |${sparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } @@ -327,7 +341,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } } - private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + private def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = { val outStream = SparkSubmit.printStream if (unknownParam != null) { outStream.println("Unknown/unsupported param " + unknownParam) From 68cb69daf3022e973422e496ccf827ca3806ff30 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 30 Oct 2014 15:54:53 -0700 Subject: [PATCH 096/115] SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop (This is just a look at what completely moving the classes would look like. I know Patrick flagged that as maybe not OK, although, it's private?) Author: Sean Owen Closes #2814 from srowen/SPARK-1209 and squashes the following commits: ead1115 [Sean Owen] Disable MIMA warnings resulting from moving the class -- this was also part of the PairRDDFunctions type hierarchy though? 2d42c1d [Sean Owen] Move SparkHadoopMapRedUtil / SparkHadoopMapReduceUtil from org.apache.hadoop to org.apache.spark --- .../main/scala/org/apache/spark/SparkHadoopWriter.scala | 1 + .../{hadoop => spark}/mapred/SparkHadoopMapRedUtil.scala | 6 ++++-- .../mapreduce/SparkHadoopMapReduceUtil.scala | 5 +++-- .../main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 + .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 3 ++- project/MimaExcludes.scala | 8 ++++++++ .../apache/spark/sql/parquet/ParquetTableOperations.scala | 1 + .../org/apache/spark/sql/hive/hiveWriterContainers.scala | 1 + 8 files changed, 21 insertions(+), 5 deletions(-) rename core/src/main/scala/org/apache/{hadoop => spark}/mapred/SparkHadoopMapRedUtil.scala (93%) rename core/src/main/scala/org/apache/{hadoop => spark}/mapreduce/SparkHadoopMapReduceUtil.scala (96%) diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 376e69cd997d5..40237596570de 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD /** diff --git a/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala similarity index 93% rename from core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala rename to core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 0c47afae54c8b..fe2bc65f89ab4 100644 --- a/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -15,9 +15,11 @@ * limitations under the License. */ -package org.apache.hadoop.mapred +package org.apache.spark.mapred -private[apache] +import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} + +private[spark] trait SparkHadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = { val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala similarity index 96% rename from core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala rename to core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala index 1fca5729c6092..3340673f91156 100644 --- a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.hadoop.mapreduce +package org.apache.spark.mapreduce import java.lang.{Boolean => JBoolean, Integer => JInteger} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID} -private[apache] +private[spark] trait SparkHadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { val klass = firstAvailableClass( diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 324563248793c..e7b11707551df 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index da89f634abaea..462f0d6268a86 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -33,13 +33,14 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, -RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} +RecordWriter => NewRecordWriter} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 6a0495f8fd540..a94d09be3bec6 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -77,6 +77,14 @@ object MimaExcludes { // SPARK-3822 ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") + ) ++ Seq( + // SPARK-1209 + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.hadoop.mapred.SparkHadoopMapRedUtil"), + ProblemFilters.exclude[MissingTypesProblem]( + "org.apache.spark.rdd.PairRDDFunctions") ) case v if v.startsWith("1.1") => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 5c6fa78ae3895..ef995b3d1a6b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -42,6 +42,7 @@ import parquet.hadoop.util.ContextUtil import parquet.io.ParquetDecodingException import parquet.schema.MessageType +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.SQLConf diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index bf2ce9df67c58..cc8bb3e172c6e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.Row import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} import org.apache.spark.sql.hive.{ShimFileSinkDesc => FileSinkDesc} From 9b6ebe33db27be38c3036ffeda17096043fb0fb9 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Thu, 30 Oct 2014 17:15:45 -0700 Subject: [PATCH 097/115] [SPARK-4120][SQL] Join of multiple tables with syntax like SELECT .. FROM T1,T2,T3.. does not work in SparkSQL Right now it works for only 2 tables like below query. sql("SELECT * FROM records1 as a,records2 as b where a.key=b.key ") But it does not work for more than 2 tables like below query sql("SELECT * FROM records1 as a,records2 as b,records3 as c where a.key=b.key and a.key=c.key"). Author: ravipesala Closes #2987 from ravipesala/multijoin and squashes the following commits: 429b005 [ravipesala] Support multiple joins --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 3 ++- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 ++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 0acf7252ba3f0..942b843104958 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -166,7 +166,8 @@ class SqlParser extends AbstractSparkSQLParser { // Based very loosely on the MySQL Grammar. // http://dev.mysql.com/doc/refman/5.0/en/join.html protected lazy val relations: Parser[LogicalPlan] = - ( relation ~ ("," ~> relation) ^^ { case r1 ~ r2 => Join(r1, r2, Inner, None) } + ( relation ~ rep1("," ~> relation) ^^ { + case r1 ~ joins => joins.foldLeft(r1) { case(lhs, r) => Join(lhs, r, Inner, None) } } | relation ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 1034c2d05f8cf..4c36ca0f41174 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -899,4 +899,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("SPARK-3814 Support Bitwise ~ operator") { checkAnswer(sql("SELECT ~key FROM testData WHERE key = 1 "), -2) } + + test("SPARK-4120 Join of multiple tables does not work in SparkSQL") { + checkAnswer( + sql( + """SELECT a.key, b.key, c.key + |FROM testData a,testData b,testData c + |where a.key = b.key and a.key = c.key + """.stripMargin), + (1 to 100).map(i => Seq(i, i, i))) + } } From 2e35e24294ad8a5e76c89ea888fe330052dabd5a Mon Sep 17 00:00:00 2001 From: Yash Datta Date: Thu, 30 Oct 2014 17:17:24 -0700 Subject: [PATCH 098/115] [SPARK-3968][SQL] Use parquet-mr filter2 api The parquet-mr project has introduced a new filter api (https://github.com/apache/incubator-parquet-mr/pull/4), along with several fixes . It can also eliminate entire RowGroups depending on certain statistics like min/max We can leverage that to further improve performance of queries with filters. Also filter2 api introduces ability to create custom filters. We can create a custom filter for the optimized In clause (InSet) , so that elimination happens in the ParquetRecordReader itself Author: Yash Datta Closes #2841 from saucam/master and squashes the following commits: 8282ba0 [Yash Datta] SPARK-3968: fix scala code style and add some more tests for filtering on optional columns 515df1c [Yash Datta] SPARK-3968: Add a test case for filter pushdown on optional column 5f4530e [Yash Datta] SPARK-3968: Fix scala code style f304667 [Yash Datta] SPARK-3968: Using task metadata strategy for row group filtering ec53e92 [Yash Datta] SPARK-3968: No push down should result in case we are unable to create a record filter 48163c3 [Yash Datta] SPARK-3968: Code cleanup cc7b596 [Yash Datta] SPARK-3968: 1. Fix RowGroupFiltering not working 2. Use the serialization/deserialization from Parquet library for filter pushdown caed851 [Yash Datta] Revert "SPARK-3968: Not pushing the filters in case of OPTIONAL columns" since filtering on optional columns is now supported in filter2 api 49703c9 [Yash Datta] SPARK-3968: Not pushing the filters in case of OPTIONAL columns 9d09741 [Yash Datta] SPARK-3968: Change parquet filter pushdown to use filter2 api of parquet-mr --- pom.xml | 2 +- .../spark/sql/parquet/ParquetFilters.scala | 230 +++++++----------- .../sql/parquet/ParquetTableOperations.scala | 179 +++++++++++--- .../spark/sql/parquet/ParquetTestData.scala | 19 ++ .../spark/sql/parquet/ParquetQuerySuite.scala | 57 +++++ 5 files changed, 308 insertions(+), 179 deletions(-) diff --git a/pom.xml b/pom.xml index e4c92470fc03e..379274d0b16e3 100644 --- a/pom.xml +++ b/pom.xml @@ -133,7 +133,7 @@ 0.13.1 10.10.1.1 - 1.4.3 + 1.6.0rc3 1.2.3 8.1.14.v20131031 0.3.6 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 7c83f1cad7d71..517a5cf0029ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -21,8 +21,12 @@ import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration -import parquet.filter._ -import parquet.filter.ColumnPredicates._ +import parquet.filter2.compat.FilterCompat +import parquet.filter2.compat.FilterCompat._ +import parquet.filter2.predicate.FilterPredicate +import parquet.filter2.predicate.FilterApi +import parquet.filter2.predicate.FilterApi._ +import parquet.io.api.Binary import parquet.column.ColumnReader import com.google.common.io.BaseEncoding @@ -38,67 +42,74 @@ private[sql] object ParquetFilters { // set this to false if pushdown should be disabled val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.hints.parquetFilterPushdown" - def createRecordFilter(filterExpressions: Seq[Expression]): UnboundRecordFilter = { + def createRecordFilter(filterExpressions: Seq[Expression]): Filter = { val filters: Seq[CatalystFilter] = filterExpressions.collect { case (expression: Expression) if createFilter(expression).isDefined => createFilter(expression).get } - if (filters.length > 0) filters.reduce(AndRecordFilter.and) else null + if (filters.length > 0) FilterCompat.get(filters.reduce(FilterApi.and)) else null } - def createFilter(expression: Expression): Option[CatalystFilter] = { + def createFilter(expression: Expression): Option[CatalystFilter] ={ def createEqualityFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case BooleanType => - ComparisonFilter.createBooleanFilter(name, literal.value.asInstanceOf[Boolean], predicate) + ComparisonFilter.createBooleanFilter( + name, + literal.value.asInstanceOf[Boolean], + predicate) case IntegerType => - ComparisonFilter.createIntFilter( + new ComparisonFilter( name, - (x: Int) => x == literal.value.asInstanceOf[Int], + FilterApi.eq(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x == literal.value.asInstanceOf[Long], + FilterApi.eq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x == literal.value.asInstanceOf[Double], + FilterApi.eq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x == literal.value.asInstanceOf[Float], + FilterApi.eq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) case StringType => - ComparisonFilter.createStringFilter(name, literal.value.asInstanceOf[String], predicate) + ComparisonFilter.createStringFilter( + name, + literal.value.asInstanceOf[String], + predicate) } + def createLessThanFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case IntegerType => - ComparisonFilter.createIntFilter( - name, - (x: Int) => x < literal.value.asInstanceOf[Int], + new ComparisonFilter( + name, + FilterApi.lt(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x < literal.value.asInstanceOf[Long], + FilterApi.lt(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x < literal.value.asInstanceOf[Double], + FilterApi.lt(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x < literal.value.asInstanceOf[Float], + FilterApi.lt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) } def createLessThanOrEqualFilter( @@ -106,24 +117,24 @@ private[sql] object ParquetFilters { literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case IntegerType => - ComparisonFilter.createIntFilter( + new ComparisonFilter( name, - (x: Int) => x <= literal.value.asInstanceOf[Int], + FilterApi.ltEq(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x <= literal.value.asInstanceOf[Long], + FilterApi.ltEq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x <= literal.value.asInstanceOf[Double], + FilterApi.ltEq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x <= literal.value.asInstanceOf[Float], + FilterApi.ltEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) } // TODO: combine these two types somehow? @@ -132,24 +143,24 @@ private[sql] object ParquetFilters { literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case IntegerType => - ComparisonFilter.createIntFilter( + new ComparisonFilter( name, - (x: Int) => x > literal.value.asInstanceOf[Int], + FilterApi.gt(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x > literal.value.asInstanceOf[Long], + FilterApi.gt(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x > literal.value.asInstanceOf[Double], + FilterApi.gt(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x > literal.value.asInstanceOf[Float], + FilterApi.gt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) } def createGreaterThanOrEqualFilter( @@ -157,23 +168,24 @@ private[sql] object ParquetFilters { literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case IntegerType => - ComparisonFilter.createIntFilter( - name, (x: Int) => x >= literal.value.asInstanceOf[Int], + new ComparisonFilter( + name, + FilterApi.gtEq(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x >= literal.value.asInstanceOf[Long], + FilterApi.gtEq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x >= literal.value.asInstanceOf[Double], + FilterApi.gtEq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x >= literal.value.asInstanceOf[Float], + FilterApi.gtEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) } @@ -209,25 +221,25 @@ private[sql] object ParquetFilters { case _ => None } } - case p @ EqualTo(left: Literal, right: NamedExpression) if !right.nullable => + case p @ EqualTo(left: Literal, right: NamedExpression) => Some(createEqualityFilter(right.name, left, p)) - case p @ EqualTo(left: NamedExpression, right: Literal) if !left.nullable => + case p @ EqualTo(left: NamedExpression, right: Literal) => Some(createEqualityFilter(left.name, right, p)) - case p @ LessThan(left: Literal, right: NamedExpression) if !right.nullable => + case p @ LessThan(left: Literal, right: NamedExpression) => Some(createLessThanFilter(right.name, left, p)) - case p @ LessThan(left: NamedExpression, right: Literal) if !left.nullable => + case p @ LessThan(left: NamedExpression, right: Literal) => Some(createLessThanFilter(left.name, right, p)) - case p @ LessThanOrEqual(left: Literal, right: NamedExpression) if !right.nullable => + case p @ LessThanOrEqual(left: Literal, right: NamedExpression) => Some(createLessThanOrEqualFilter(right.name, left, p)) - case p @ LessThanOrEqual(left: NamedExpression, right: Literal) if !left.nullable => + case p @ LessThanOrEqual(left: NamedExpression, right: Literal) => Some(createLessThanOrEqualFilter(left.name, right, p)) - case p @ GreaterThan(left: Literal, right: NamedExpression) if !right.nullable => + case p @ GreaterThan(left: Literal, right: NamedExpression) => Some(createGreaterThanFilter(right.name, left, p)) - case p @ GreaterThan(left: NamedExpression, right: Literal) if !left.nullable => + case p @ GreaterThan(left: NamedExpression, right: Literal) => Some(createGreaterThanFilter(left.name, right, p)) - case p @ GreaterThanOrEqual(left: Literal, right: NamedExpression) if !right.nullable => + case p @ GreaterThanOrEqual(left: Literal, right: NamedExpression) => Some(createGreaterThanOrEqualFilter(right.name, left, p)) - case p @ GreaterThanOrEqual(left: NamedExpression, right: Literal) if !left.nullable => + case p @ GreaterThanOrEqual(left: NamedExpression, right: Literal) => Some(createGreaterThanOrEqualFilter(left.name, right, p)) case _ => None } @@ -300,52 +312,54 @@ private[sql] object ParquetFilters { } abstract private[parquet] class CatalystFilter( - @transient val predicate: CatalystPredicate) extends UnboundRecordFilter + @transient val predicate: CatalystPredicate) extends FilterPredicate private[parquet] case class ComparisonFilter( val columnName: String, - private var filter: UnboundRecordFilter, + private var filter: FilterPredicate, @transient override val predicate: CatalystPredicate) extends CatalystFilter(predicate) { - override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { - filter.bind(readers) + override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { + filter.accept(visitor) } } private[parquet] case class OrFilter( - private var filter: UnboundRecordFilter, + private var filter: FilterPredicate, @transient val left: CatalystFilter, @transient val right: CatalystFilter, @transient override val predicate: Or) extends CatalystFilter(predicate) { def this(l: CatalystFilter, r: CatalystFilter) = this( - OrRecordFilter.or(l, r), + FilterApi.or(l, r), l, r, Or(l.predicate, r.predicate)) - override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { - filter.bind(readers) + override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { + filter.accept(visitor); } + } private[parquet] case class AndFilter( - private var filter: UnboundRecordFilter, + private var filter: FilterPredicate, @transient val left: CatalystFilter, @transient val right: CatalystFilter, @transient override val predicate: And) extends CatalystFilter(predicate) { def this(l: CatalystFilter, r: CatalystFilter) = this( - AndRecordFilter.and(l, r), + FilterApi.and(l, r), l, r, And(l.predicate, r.predicate)) - override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { - filter.bind(readers) + override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { + filter.accept(visitor); } + } private[parquet] object ComparisonFilter { @@ -355,13 +369,7 @@ private[parquet] object ComparisonFilter { predicate: CatalystPredicate): CatalystFilter = new ComparisonFilter( columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToBoolean( - new BooleanPredicateFunction { - def functionToApply(input: Boolean): Boolean = input == value - } - )), + FilterApi.eq(booleanColumn(columnName), value.asInstanceOf[java.lang.Boolean]), predicate) def createStringFilter( @@ -370,72 +378,6 @@ private[parquet] object ComparisonFilter { predicate: CatalystPredicate): CatalystFilter = new ComparisonFilter( columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToString ( - new ColumnPredicates.PredicateFunction[String] { - def functionToApply(input: String): Boolean = input == value - } - )), - predicate) - - def createIntFilter( - columnName: String, - func: Int => Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToInteger( - new IntegerPredicateFunction { - def functionToApply(input: Int) = func(input) - } - )), - predicate) - - def createLongFilter( - columnName: String, - func: Long => Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToLong( - new LongPredicateFunction { - def functionToApply(input: Long) = func(input) - } - )), - predicate) - - def createDoubleFilter( - columnName: String, - func: Double => Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToDouble( - new DoublePredicateFunction { - def functionToApply(input: Double) = func(input) - } - )), - predicate) - - def createFloatFilter( - columnName: String, - func: Float => Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToFloat( - new FloatPredicateFunction { - def functionToApply(input: Float) = func(input) - } - )), + FilterApi.eq(binaryColumn(columnName), Binary.fromString(value)), predicate) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index ef995b3d1a6b4..416bf5614437f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -38,6 +38,7 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import parquet.hadoop._ import parquet.hadoop.api.{InitContext, ReadSupport} import parquet.hadoop.metadata.GlobalMetaData +import parquet.hadoop.api.ReadSupport.ReadContext import parquet.hadoop.util.ContextUtil import parquet.io.ParquetDecodingException import parquet.schema.MessageType @@ -77,6 +78,10 @@ case class ParquetTableScan( s"$normalOutput + $partOutput != $attributes, ${relation.output}") override def execute(): RDD[Row] = { + import parquet.filter2.compat.FilterCompat.FilterPredicateCompat + import parquet.filter2.compat.FilterCompat.Filter + import parquet.filter2.predicate.FilterPredicate + val sc = sqlContext.sparkContext val job = new Job(sc.hadoopConfiguration) ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) @@ -107,7 +112,13 @@ case class ParquetTableScan( // "spark.sql.hints.parquetFilterPushdown" to false inside SparkConf. if (columnPruningPred.length > 0 && sc.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) { - ParquetFilters.serializeFilterExpressions(columnPruningPred, conf) + + // Set this in configuration of ParquetInputFormat, needed for RowGroupFiltering + val filter: Filter = ParquetFilters.createRecordFilter(columnPruningPred) + if (filter != null){ + val filterPredicate = filter.asInstanceOf[FilterPredicateCompat].getFilterPredicate() + ParquetInputFormat.setFilterPredicate(conf, filterPredicate) + } } // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata @@ -363,15 +374,17 @@ private[parquet] class FilteringParquetRowInputFormat override def createRecordReader( inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = { + + import parquet.filter2.compat.FilterCompat.NoOpFilter + import parquet.filter2.compat.FilterCompat.Filter + val readSupport: ReadSupport[Row] = new RowReadSupport() - val filterExpressions = - ParquetFilters.deserializeFilterExpressions(ContextUtil.getConfiguration(taskAttemptContext)) - if (filterExpressions.length > 0) { - logInfo(s"Pushing down predicates for RecordFilter: ${filterExpressions.mkString(", ")}") + val filter = ParquetInputFormat.getFilter(ContextUtil.getConfiguration(taskAttemptContext)) + if (!filter.isInstanceOf[NoOpFilter]) { new ParquetRecordReader[Row]( readSupport, - ParquetFilters.createRecordFilter(filterExpressions)) + filter) } else { new ParquetRecordReader[Row](readSupport) } @@ -424,10 +437,8 @@ private[parquet] class FilteringParquetRowInputFormat configuration: Configuration, footers: JList[Footer]): JList[ParquetInputSplit] = { - import FilteringParquetRowInputFormat.blockLocationCache - - val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) - + // Use task side strategy by default + val taskSideMetaData = configuration.getBoolean(ParquetInputFormat.TASK_SIDE_METADATA, true) val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue) val minSplitSize: JLong = Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L)) @@ -436,23 +447,67 @@ private[parquet] class FilteringParquetRowInputFormat s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" + s" minSplitSize = $minSplitSize") } - val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + + // Uses strict type checking by default val getGlobalMetaData = classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]]) getGlobalMetaData.setAccessible(true) val globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData] - // if parquet file is empty, return empty splits. - if (globalMetaData == null) { - return splits - } + if (globalMetaData == null) { + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + return splits + } + val readContext = getReadSupport(configuration).init( new InitContext(configuration, globalMetaData.getKeyValueMetaData(), globalMetaData.getSchema())) + + if (taskSideMetaData){ + logInfo("Using Task Side Metadata Split Strategy") + return getTaskSideSplits(configuration, + footers, + maxSplitSize, + minSplitSize, + readContext) + } else { + logInfo("Using Client Side Metadata Split Strategy") + return getClientSideSplits(configuration, + footers, + maxSplitSize, + minSplitSize, + readContext) + } + + } + + def getClientSideSplits( + configuration: Configuration, + footers: JList[Footer], + maxSplitSize: JLong, + minSplitSize: JLong, + readContext: ReadContext): JList[ParquetInputSplit] = { + + import FilteringParquetRowInputFormat.blockLocationCache + import parquet.filter2.compat.FilterCompat; + import parquet.filter2.compat.FilterCompat.Filter; + import parquet.filter2.compat.RowGroupFilter; + + val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + val filter: Filter = ParquetInputFormat.getFilter(configuration) + var rowGroupsDropped: Long = 0 + var totalRowGroups: Long = 0 + + // Ugly hack, stuck with it until PR: + // https://github.com/apache/incubator-parquet-mr/pull/17 + // is resolved val generateSplits = - classOf[ParquetInputFormat[_]].getDeclaredMethods.find(_.getName == "generateSplits").get + Class.forName("parquet.hadoop.ClientSideMetadataSplitStrategy") + .getDeclaredMethods.find(_.getName == "generateSplits").getOrElse( + sys.error(s"Failed to reflectively invoke ClientSideMetadataSplitStrategy.generateSplits")) generateSplits.setAccessible(true) for (footer <- footers) { @@ -461,29 +516,85 @@ private[parquet] class FilteringParquetRowInputFormat val status = fileStatuses.getOrElse(file, fs.getFileStatus(file)) val parquetMetaData = footer.getParquetMetadata val blocks = parquetMetaData.getBlocks - var blockLocations: Array[BlockLocation] = null - if (!cacheMetadata) { - blockLocations = fs.getFileBlockLocations(status, 0, status.getLen) - } else { - blockLocations = blockLocationCache.get(status, new Callable[Array[BlockLocation]] { - def call(): Array[BlockLocation] = fs.getFileBlockLocations(status, 0, status.getLen) - }) - } + totalRowGroups = totalRowGroups + blocks.size + val filteredBlocks = RowGroupFilter.filterRowGroups( + filter, + blocks, + parquetMetaData.getFileMetaData.getSchema) + rowGroupsDropped = rowGroupsDropped + (blocks.size - filteredBlocks.size) + + if (!filteredBlocks.isEmpty){ + var blockLocations: Array[BlockLocation] = null + if (!cacheMetadata) { + blockLocations = fs.getFileBlockLocations(status, 0, status.getLen) + } else { + blockLocations = blockLocationCache.get(status, new Callable[Array[BlockLocation]] { + def call(): Array[BlockLocation] = fs.getFileBlockLocations(status, 0, status.getLen) + }) + } + splits.addAll( + generateSplits.invoke( + null, + filteredBlocks, + blockLocations, + status, + readContext.getRequestedSchema.toString, + readContext.getReadSupportMetadata, + minSplitSize, + maxSplitSize).asInstanceOf[JList[ParquetInputSplit]]) + } + } + + if (rowGroupsDropped > 0 && totalRowGroups > 0){ + val percentDropped = ((rowGroupsDropped/totalRowGroups.toDouble) * 100).toInt + logInfo(s"Dropping $rowGroupsDropped row groups that do not pass filter predicate " + + s"($percentDropped %) !") + } + else { + logInfo("There were no row groups that could be dropped due to filter predicates") + } + splits + + } + + def getTaskSideSplits( + configuration: Configuration, + footers: JList[Footer], + maxSplitSize: JLong, + minSplitSize: JLong, + readContext: ReadContext): JList[ParquetInputSplit] = { + + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + + // Ugly hack, stuck with it until PR: + // https://github.com/apache/incubator-parquet-mr/pull/17 + // is resolved + val generateSplits = + Class.forName("parquet.hadoop.TaskSideMetadataSplitStrategy") + .getDeclaredMethods.find(_.getName == "generateTaskSideMDSplits").getOrElse( + sys.error( + s"Failed to reflectively invoke TaskSideMetadataSplitStrategy.generateTaskSideMDSplits")) + generateSplits.setAccessible(true) + + for (footer <- footers) { + val file = footer.getFile + val fs = file.getFileSystem(configuration) + val status = fileStatuses.getOrElse(file, fs.getFileStatus(file)) + val blockLocations = fs.getFileBlockLocations(status, 0, status.getLen) splits.addAll( generateSplits.invoke( - null, - blocks, - blockLocations, - status, - parquetMetaData.getFileMetaData, - readContext.getRequestedSchema.toString, - readContext.getReadSupportMetadata, - minSplitSize, - maxSplitSize).asInstanceOf[JList[ParquetInputSplit]]) + null, + blockLocations, + status, + readContext.getRequestedSchema.toString, + readContext.getReadSupportMetadata, + minSplitSize, + maxSplitSize).asInstanceOf[JList[ParquetInputSplit]]) } splits - } + } + } private[parquet] object FilteringParquetRowInputFormat { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 837ea7695dbb3..c0918a40d136f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -92,6 +92,12 @@ private[sql] object ParquetTestData { required int64 mylong; required float myfloat; required double mydouble; + optional boolean myoptboolean; + optional int32 myoptint; + optional binary myoptstring (UTF8); + optional int64 myoptlong; + optional float myoptfloat; + optional double myoptdouble; } """ @@ -255,6 +261,19 @@ private[sql] object ParquetTestData { record.add(3, i.toLong) record.add(4, i.toFloat + 0.5f) record.add(5, i.toDouble + 0.5d) + if (i % 2 == 0) { + if (i % 3 == 0) { + record.add(6, true) + } else { + record.add(6, false) + } + record.add(7, i) + record.add(8, i.toString) + record.add(9, i.toLong) + record.add(10, i.toFloat + 0.5f) + record.add(11, i.toDouble + 0.5d) + } + writer.write(record) } writer.close() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 25e41ecf28e2e..9979ab446d8b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -560,6 +560,63 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(stringResult.size === 1) assert(stringResult(0).getString(2) == "100", "stringvalue incorrect") assert(stringResult(0).getInt(1) === 100) + + val query7 = sql(s"SELECT * FROM testfiltersource WHERE myoptint < 40") + assert( + query7.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val optResult = query7.collect() + assert(optResult.size === 20) + for(i <- 0 until 20) { + if (optResult(i)(7) != i * 2) { + fail(s"optional Int value in result row $i should be ${2*4*i}") + } + } + for(myval <- Seq("myoptint", "myoptlong", "myoptdouble", "myoptfloat")) { + val query8 = sql(s"SELECT * FROM testfiltersource WHERE $myval < 150 AND $myval >= 100") + assert( + query8.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result8 = query8.collect() + assert(result8.size === 25) + assert(result8(0)(7) === 100) + assert(result8(24)(7) === 148) + val query9 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 150 AND $myval <= 200") + assert( + query9.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result9 = query9.collect() + assert(result9.size === 25) + if (myval == "myoptint" || myval == "myoptlong") { + assert(result9(0)(7) === 152) + assert(result9(24)(7) === 200) + } else { + assert(result9(0)(7) === 150) + assert(result9(24)(7) === 198) + } + } + val query10 = sql("SELECT * FROM testfiltersource WHERE myoptstring = \"100\"") + assert( + query10.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result10 = query10.collect() + assert(result10.size === 1) + assert(result10(0).getString(8) == "100", "stringvalue incorrect") + assert(result10(0).getInt(7) === 100) + val query11 = sql(s"SELECT * FROM testfiltersource WHERE myoptboolean = true AND myoptint < 40") + assert( + query11.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result11 = query11.collect() + assert(result11.size === 7) + for(i <- 0 until 6) { + if (!result11(i).getBoolean(6)) { + fail(s"optional Boolean value in result row $i not true") + } + if (result11(i).getInt(7) != i * 6) { + fail(s"optional Int value in result row $i should be ${6*i}") + } + } } test("SPARK-1913 regression: columns only referenced by pushed down filters should remain") { From 26d31d15fda3f63707a28d1a1115770ad127cf8f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 17:56:10 -0700 Subject: [PATCH 099/115] Revert "SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop" This reverts commit 68cb69daf3022e973422e496ccf827ca3806ff30. --- .../{spark => hadoop}/mapred/SparkHadoopMapRedUtil.scala | 6 ++---- .../mapreduce/SparkHadoopMapReduceUtil.scala | 5 ++--- .../main/scala/org/apache/spark/SparkHadoopWriter.scala | 1 - .../main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 - .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 3 +-- project/MimaExcludes.scala | 8 -------- .../apache/spark/sql/parquet/ParquetTableOperations.scala | 1 - .../org/apache/spark/sql/hive/hiveWriterContainers.scala | 1 - 8 files changed, 5 insertions(+), 21 deletions(-) rename core/src/main/scala/org/apache/{spark => hadoop}/mapred/SparkHadoopMapRedUtil.scala (93%) rename core/src/main/scala/org/apache/{spark => hadoop}/mapreduce/SparkHadoopMapReduceUtil.scala (96%) diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala rename to core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala index fe2bc65f89ab4..0c47afae54c8b 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.mapred +package org.apache.hadoop.mapred -import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} - -private[spark] +private[apache] trait SparkHadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = { val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala rename to core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala index 3340673f91156..1fca5729c6092 100644 --- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.spark.mapreduce +package org.apache.hadoop.mapreduce import java.lang.{Boolean => JBoolean, Integer => JInteger} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID} -private[spark] +private[apache] trait SparkHadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { val klass = firstAvailableClass( diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 40237596570de..376e69cd997d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path -import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD /** diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index e7b11707551df..324563248793c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -35,7 +35,6 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 462f0d6268a86..da89f634abaea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -33,14 +33,13 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, -RecordWriter => NewRecordWriter} +RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a94d09be3bec6..6a0495f8fd540 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -77,14 +77,6 @@ object MimaExcludes { // SPARK-3822 ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") - ) ++ Seq( - // SPARK-1209 - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.hadoop.mapred.SparkHadoopMapRedUtil"), - ProblemFilters.exclude[MissingTypesProblem]( - "org.apache.spark.rdd.PairRDDFunctions") ) case v if v.startsWith("1.1") => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 416bf5614437f..9664c565a0b86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -43,7 +43,6 @@ import parquet.hadoop.util.ContextUtil import parquet.io.ParquetDecodingException import parquet.schema.MessageType -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.SQLConf diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index cc8bb3e172c6e..bf2ce9df67c58 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -31,7 +31,6 @@ import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ -import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.Row import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} import org.apache.spark.sql.hive.{ShimFileSinkDesc => FileSinkDesc} From 0734d09320fe37edd3a02718511cda0bda852478 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 30 Oct 2014 20:15:36 -0700 Subject: [PATCH 100/115] HOTFIX: Clean up build in network module. This is currently breaking the package build for some people (including me). This patch does some general clean-up which also fixes the current issue. - Uses consistent artifact naming - Adds sbt support for this module - Changes tests to use scalatest (fixes the original issue[1]) One thing to note, it turns out that scalatest when invoked in the Maven build doesn't succesfully detect JUnit Java tests. This is a long standing issue, I noticed it applies to all of our current test suites as well. I've created SPARK-4159 to fix this. [1] The original issue is that we need to allocate extra memory for the tests, happens by default in our scalatest configuration. Author: Patrick Wendell Closes #3025 from pwendell/hotfix and squashes the following commits: faa9053 [Patrick Wendell] HOTFIX: Clean up build in network module. --- core/pom.xml | 2 +- network/common/pom.xml | 34 +++++++++++++++++----------------- project/SparkBuild.scala | 8 +++++--- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8020a2daf81ec..6963ce4777e6f 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -46,7 +46,7 @@ org.apache.spark - network + spark-network-common_2.10 ${project.version} diff --git a/network/common/pom.xml b/network/common/pom.xml index e3b7e328701b4..a33e44b63d981 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -27,12 +27,12 @@ org.apache.spark - network + spark-network-common_2.10 jar - Shuffle Streaming Service + Spark Project Common Network Code http://spark.apache.org/ - network + network-common @@ -59,6 +59,11 @@ junit test + + com.novocode + junit-interface + test + log4j log4j @@ -69,25 +74,20 @@ mockito-all test + + org.scalatest + scalatest_${scala.binary.version} + test + - - target/java/classes - target/java/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes - org.apache.maven.plugins - maven-surefire-plugin - 2.17 - - false - - **/Test*.java - **/*Test.java - **/*Suite.java - - + org.scalatest + scalatest-maven-plugin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6d5eb681c6131..77083518bbab3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -31,10 +31,10 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, - sql, streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, + sql, networkCommon, streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", + "sql", "network-common", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = @@ -142,7 +142,9 @@ object SparkBuild extends PomBuild { // TODO: Add Sql to mima checks allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - streamingFlumeSink).contains(x)).foreach(x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) + streamingFlumeSink, networkCommon).contains(x)).foreach { + x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) + } /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) From 872fc669b497fb255db3212568f2a14c2ba0d5db Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 30 Oct 2014 22:25:18 -0700 Subject: [PATCH 101/115] [SPARK-4124] [MLlib] [PySpark] simplify serialization in MLlib Python API Create several helper functions to call MLlib Java API, convert the arguments to Java type and convert return value to Python object automatically, this simplify serialization in MLlib Python API very much. After this, the MLlib Python API does not need to deal with serialization details anymore, it's easier to add new API. cc mengxr Author: Davies Liu Closes #2995 from davies/cleanup and squashes the following commits: 8fa6ec6 [Davies Liu] address comments 16b85a0 [Davies Liu] Merge branch 'master' of github.com:apache/spark into cleanup 43743e5 [Davies Liu] bugfix 731331f [Davies Liu] simplify serialization in MLlib Python API --- .../mllib/api/python/PythonMLLibAPI.scala | 84 ++++++----- python/pyspark/mllib/classification.py | 30 ++-- python/pyspark/mllib/clustering.py | 15 +- python/pyspark/mllib/common.py | 135 ++++++++++++++++++ python/pyspark/mllib/feature.py | 122 +++------------- python/pyspark/mllib/linalg.py | 12 -- python/pyspark/mllib/random.py | 34 ++--- python/pyspark/mllib/recommendation.py | 62 ++------ python/pyspark/mllib/regression.py | 52 +++---- python/pyspark/mllib/stat.py | 65 ++------- python/pyspark/mllib/tree.py | 55 ++----- python/pyspark/mllib/util.py | 7 +- 12 files changed, 287 insertions(+), 386 deletions(-) create mode 100644 python/pyspark/mllib/common.py 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 485abe272326c..acdc67ddc660a 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 @@ -18,7 +18,7 @@ package org.apache.spark.mllib.api.python import java.io.OutputStream -import java.util.{ArrayList => JArrayList} +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.language.existentials @@ -72,15 +72,11 @@ class PythonMLLibAPI extends Serializable { private def trainRegressionModel( learner: GeneralizedLinearAlgorithm[_ <: GeneralizedLinearModel], data: JavaRDD[LabeledPoint], - initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { - val initialWeights = SerDe.loads(initialWeightsBA).asInstanceOf[Vector] + initialWeights: Vector): JList[Object] = { // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. learner.disableUncachedWarning() val model = learner.run(data.rdd, initialWeights) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.dumps(model.weights)) - ret.add(model.intercept: java.lang.Double) - ret + List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava } /** @@ -91,10 +87,10 @@ class PythonMLLibAPI extends Serializable { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte], + initialWeights: Vector, regParam: Double, regType: String, - intercept: Boolean): java.util.List[java.lang.Object] = { + intercept: Boolean): JList[Object] = { val lrAlg = new LinearRegressionWithSGD() lrAlg.setIntercept(intercept) lrAlg.optimizer @@ -113,7 +109,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( lrAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -125,7 +121,7 @@ class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + initialWeights: Vector): JList[Object] = { val lassoAlg = new LassoWithSGD() lassoAlg.optimizer .setNumIterations(numIterations) @@ -135,7 +131,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( lassoAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -147,7 +143,7 @@ class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + initialWeights: Vector): JList[Object] = { val ridgeAlg = new RidgeRegressionWithSGD() ridgeAlg.optimizer .setNumIterations(numIterations) @@ -157,7 +153,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( ridgeAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -169,9 +165,9 @@ class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte], + initialWeights: Vector, regType: String, - intercept: Boolean): java.util.List[java.lang.Object] = { + intercept: Boolean): JList[Object] = { val SVMAlg = new SVMWithSGD() SVMAlg.setIntercept(intercept) SVMAlg.optimizer @@ -190,7 +186,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( SVMAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -201,10 +197,10 @@ class PythonMLLibAPI extends Serializable { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte], + initialWeights: Vector, regParam: Double, regType: String, - intercept: Boolean): java.util.List[java.lang.Object] = { + intercept: Boolean): JList[Object] = { val LogRegAlg = new LogisticRegressionWithSGD() LogRegAlg.setIntercept(intercept) LogRegAlg.optimizer @@ -223,7 +219,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( LogRegAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -231,13 +227,10 @@ class PythonMLLibAPI extends Serializable { */ def trainNaiveBayes( data: JavaRDD[LabeledPoint], - lambda: Double): java.util.List[java.lang.Object] = { + lambda: Double): JList[Object] = { val model = NaiveBayes.train(data.rdd, lambda) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(Vectors.dense(model.labels)) - ret.add(Vectors.dense(model.pi)) - ret.add(model.theta) - ret + List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta). + map(_.asInstanceOf[Object]).asJava } /** @@ -259,6 +252,21 @@ class PythonMLLibAPI extends Serializable { return kMeansAlg.run(data.rdd) } + /** + * A Wrapper of MatrixFactorizationModel to provide helpfer method for Python + */ + private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorizationModel) + extends MatrixFactorizationModel(model.rank, model.userFeatures, model.productFeatures) { + + def predict(userAndProducts: JavaRDD[Array[Any]]): RDD[Rating] = + predict(SerDe.asTupleRDD(userAndProducts.rdd)) + + def getUserFeatures = SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) + + def getProductFeatures = SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) + + } + /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care @@ -271,7 +279,7 @@ class PythonMLLibAPI extends Serializable { iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = { - ALS.train(ratings.rdd, rank, iterations, lambda, blocks) + new MatrixFactorizationModelWrapper(ALS.train(ratings.rdd, rank, iterations, lambda, blocks)) } /** @@ -287,7 +295,8 @@ class PythonMLLibAPI extends Serializable { lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = { - ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha) + new MatrixFactorizationModelWrapper( + ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha)) } /** @@ -373,19 +382,16 @@ class PythonMLLibAPI extends Serializable { rdd.rdd.map(model.transform) } - def findSynonyms(word: String, num: Int): java.util.List[java.lang.Object] = { + def findSynonyms(word: String, num: Int): JList[Object] = { val vec = transform(word) findSynonyms(vec, num) } - def findSynonyms(vector: Vector, num: Int): java.util.List[java.lang.Object] = { + def findSynonyms(vector: Vector, num: Int): JList[Object] = { val result = model.findSynonyms(vector, num) val similarity = Vectors.dense(result.map(_._2)) val words = result.map(_._1) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(words) - ret.add(similarity) - ret + List(words, similarity).map(_.asInstanceOf[Object]).asJava } } @@ -395,13 +401,13 @@ class PythonMLLibAPI extends Serializable { * Extra care needs to be taken in the Python code to ensure it gets freed on exit; * see the Py4J documentation. * @param data Training data - * @param categoricalFeaturesInfoJMap Categorical features info, as Java map + * @param categoricalFeaturesInfo Categorical features info, as Java map */ def trainDecisionTreeModel( data: JavaRDD[LabeledPoint], algoStr: String, numClasses: Int, - categoricalFeaturesInfoJMap: java.util.Map[Int, Int], + categoricalFeaturesInfo: JMap[Int, Int], impurityStr: String, maxDepth: Int, maxBins: Int, @@ -417,7 +423,7 @@ class PythonMLLibAPI extends Serializable { maxDepth = maxDepth, numClassesForClassification = numClasses, maxBins = maxBins, - categoricalFeaturesInfo = categoricalFeaturesInfoJMap.asScala.toMap, + categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap, minInstancesPerNode = minInstancesPerNode, minInfoGain = minInfoGain) @@ -589,7 +595,7 @@ private[spark] object SerDe extends Serializable { if (objects.length == 0 || objects.length > 3) { out.write(Opcodes.MARK) } - objects.foreach(pickler.save(_)) + objects.foreach(pickler.save) val code = objects.length match { case 1 => Opcodes.TUPLE1 case 2 => Opcodes.TUPLE2 @@ -719,7 +725,7 @@ private[spark] object SerDe extends Serializable { } /* convert RDD[Tuple2[,]] to RDD[Array[Any]] */ - def fromTuple2RDD(rdd: RDD[Tuple2[Any, Any]]): RDD[Array[Any]] = { + def fromTuple2RDD(rdd: RDD[(Any, Any)]): RDD[Array[Any]] = { rdd.map(x => Array(x._1, x._2)) } diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index e295c9d0954d9..297a2bf37d2cf 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -20,8 +20,8 @@ import numpy from numpy import array -from pyspark import SparkContext, PickleSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc +from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper @@ -102,14 +102,11 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, training data (i.e. whether bias features are activated or not). """ - sc = data.context + def train(rdd, i): + return callMLlibFunc("trainLogisticRegressionModelWithSGD", rdd, iterations, step, + miniBatchFraction, i, regParam, regType, intercept) - def train(jdata, i): - return sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( - jdata, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - - return _regression_train_wrapper(sc, train, LogisticRegressionModel, data, - initialWeights) + return _regression_train_wrapper(train, LogisticRegressionModel, data, initialWeights) class SVMModel(LinearModel): @@ -174,13 +171,11 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, training data (i.e. whether bias features are activated or not). """ - sc = data.context - - def train(jrdd, i): - return sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( - jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) + def train(rdd, i): + return callMLlibFunc("trainSVMModelWithSGD", rdd, iterations, step, regParam, + miniBatchFraction, i, regType, intercept) - return _regression_train_wrapper(sc, train, SVMModel, data, initialWeights) + return _regression_train_wrapper(train, SVMModel, data, initialWeights) class NaiveBayesModel(object): @@ -243,14 +238,13 @@ def train(cls, data, lambda_=1.0): (e.g. a count vector). :param lambda_: The smoothing parameter """ - sc = data.context - jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(_to_java_object_rdd(data), lambda_) - labels, pi, theta = PickleSerializer().loads(str(sc._jvm.SerDe.dumps(jlist))) + labels, pi, theta = callMLlibFunc("trainNaiveBayes", data, lambda_) return NaiveBayesModel(labels.toArray(), pi.toArray(), numpy.array(theta)) def _test(): import doctest + from pyspark import SparkContext globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 5ee7997104d21..fe4c4cc5094d8 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -16,8 +16,8 @@ # from pyspark import SparkContext -from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, callJavaFunc, _to_java_object_rdd +from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['KMeansModel', 'KMeans'] @@ -80,14 +80,11 @@ class KMeans(object): @classmethod def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" - sc = rdd.context - ser = PickleSerializer() # cache serialized data to avoid objects over head in JVM - cached = rdd.map(_convert_to_vector)._reserialize(AutoBatchedSerializer(ser)).cache() - model = sc._jvm.PythonMLLibAPI().trainKMeansModel( - _to_java_object_rdd(cached), k, maxIterations, runs, initializationMode) - bytes = sc._jvm.SerDe.dumps(model.clusterCenters()) - centers = ser.loads(str(bytes)) + jcached = _to_java_object_rdd(rdd.map(_convert_to_vector), cache=True) + model = callMLlibFunc("trainKMeansModel", jcached, k, maxIterations, runs, + initializationMode) + centers = callJavaFunc(rdd.context, model.clusterCenters) return KMeansModel([c.toArray() for c in centers]) diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py new file mode 100644 index 0000000000000..76864d8163586 --- /dev/null +++ b/python/pyspark/mllib/common.py @@ -0,0 +1,135 @@ +# +# 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. +# + +import py4j.protocol +from py4j.protocol import Py4JJavaError +from py4j.java_gateway import JavaObject +from py4j.java_collections import MapConverter, ListConverter, JavaArray, JavaList + +from pyspark import RDD, SparkContext +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer + + +# Hack for support float('inf') in Py4j +_old_smart_decode = py4j.protocol.smart_decode + +_float_str_mapping = { + 'nan': 'NaN', + 'inf': 'Infinity', + '-inf': '-Infinity', +} + + +def _new_smart_decode(obj): + if isinstance(obj, float): + s = unicode(obj) + return _float_str_mapping.get(s, s) + return _old_smart_decode(obj) + +py4j.protocol.smart_decode = _new_smart_decode + + +_picklable_classes = [ + 'LinkedList', + 'SparseVector', + 'DenseVector', + 'DenseMatrix', + 'Rating', + 'LabeledPoint', +] + + +# this will call the MLlib version of pythonToJava() +def _to_java_object_rdd(rdd, cache=False): + """ Return an JavaRDD of Object by unpickling + + It will convert each Python object into Java object by Pyrolite, whenever the + RDD is serialized in batch or not. + """ + rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) + if cache: + rdd.cache() + return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) + + +def _py2java(sc, obj): + """ Convert Python object into Java """ + if isinstance(obj, RDD): + obj = _to_java_object_rdd(obj) + elif isinstance(obj, SparkContext): + obj = obj._jsc + elif isinstance(obj, dict): + obj = MapConverter().convert(obj, sc._gateway._gateway_client) + elif isinstance(obj, (list, tuple)): + obj = ListConverter().convert(obj, sc._gateway._gateway_client) + elif isinstance(obj, JavaObject): + pass + elif isinstance(obj, (int, long, float, bool, basestring)): + pass + else: + bytes = bytearray(PickleSerializer().dumps(obj)) + obj = sc._jvm.SerDe.loads(bytes) + return obj + + +def _java2py(sc, r): + if isinstance(r, JavaObject): + clsName = r.getClass().getSimpleName() + # convert RDD into JavaRDD + if clsName != 'JavaRDD' and clsName.endswith("RDD"): + r = r.toJavaRDD() + clsName = 'JavaRDD' + + if clsName == 'JavaRDD': + jrdd = sc._jvm.SerDe.javaToPython(r) + return RDD(jrdd, sc, AutoBatchedSerializer(PickleSerializer())) + + elif isinstance(r, (JavaArray, JavaList)) or clsName in _picklable_classes: + r = sc._jvm.SerDe.dumps(r) + + if isinstance(r, bytearray): + r = PickleSerializer().loads(str(r)) + return r + + +def callJavaFunc(sc, func, *args): + """ Call Java Function """ + args = [_py2java(sc, a) for a in args] + return _java2py(sc, func(*args)) + + +def callMLlibFunc(name, *args): + """ Call API in PythonMLLibAPI """ + sc = SparkContext._active_spark_context + api = getattr(sc._jvm.PythonMLLibAPI(), name) + return callJavaFunc(sc, api, *args) + + +class JavaModelWrapper(object): + """ + Wrapper for the model in JVM + """ + def __init__(self, java_model): + self._sc = SparkContext._active_spark_context + self._java_model = java_model + + def __del__(self): + self._sc._gateway.detach(self._java_model) + + def call(self, name, *a): + """Call method of java_model""" + return callJavaFunc(self._sc, getattr(self._java_model, name), *a) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 324343443ebdb..44bf6f269d7a3 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -21,89 +21,16 @@ import sys import warnings -import py4j.protocol from py4j.protocol import Py4JJavaError -from py4j.java_gateway import JavaObject from pyspark import RDD, SparkContext -from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import Vectors, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper +from pyspark.mllib.linalg import Vectors __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] -# Hack for support float('inf') in Py4j -_old_smart_decode = py4j.protocol.smart_decode - -_float_str_mapping = { - u'nan': u'NaN', - u'inf': u'Infinity', - u'-inf': u'-Infinity', -} - - -def _new_smart_decode(obj): - if isinstance(obj, float): - s = unicode(obj) - return _float_str_mapping.get(s, s) - return _old_smart_decode(obj) - -py4j.protocol.smart_decode = _new_smart_decode - - -# TODO: move these helper functions into utils -_picklable_classes = [ - 'LinkedList', - 'SparseVector', - 'DenseVector', - 'DenseMatrix', - 'Rating', - 'LabeledPoint', -] - - -def _py2java(sc, a): - """ Convert Python object into Java """ - if isinstance(a, RDD): - a = _to_java_object_rdd(a) - elif not isinstance(a, (int, long, float, bool, basestring)): - bytes = bytearray(PickleSerializer().dumps(a)) - a = sc._jvm.SerDe.loads(bytes) - return a - - -def _java2py(sc, r): - if isinstance(r, JavaObject): - clsName = r.getClass().getSimpleName() - if clsName in ("RDD", "JavaRDD"): - if clsName == "RDD": - r = r.toJavaRDD() - jrdd = sc._jvm.SerDe.javaToPython(r) - return RDD(jrdd, sc, AutoBatchedSerializer(PickleSerializer())) - - elif clsName in _picklable_classes: - r = sc._jvm.SerDe.dumps(r) - - if isinstance(r, bytearray): - r = PickleSerializer().loads(str(r)) - return r - - -def _callJavaFunc(sc, func, *args): - """ Call Java Function - """ - args = [_py2java(sc, a) for a in args] - return _java2py(sc, func(*args)) - - -def _callAPI(sc, name, *args): - """ Call API in PythonMLLibAPI - """ - api = getattr(sc._jvm.PythonMLLibAPI(), name) - return _callJavaFunc(sc, api, *args) - - class VectorTransformer(object): """ :: DeveloperApi :: @@ -160,25 +87,19 @@ def transform(self, vector): """ sc = SparkContext._active_spark_context assert sc is not None, "SparkContext should be initialized first" - return _callAPI(sc, "normalizeVector", self.p, vector) + return callMLlibFunc("normalizeVector", self.p, vector) -class JavaModelWrapper(VectorTransformer): +class JavaVectorTransformer(JavaModelWrapper, VectorTransformer): """ Wrapper for the model in JVM """ - def __init__(self, sc, java_model): - self._sc = sc - self._java_model = java_model - - def __del__(self): - self._sc._gateway.detach(self._java_model) def transform(self, dataset): - return _callJavaFunc(self._sc, self._java_model.transform, dataset) + return self.call("transform", dataset) -class StandardScalerModel(JavaModelWrapper): +class StandardScalerModel(JavaVectorTransformer): """ :: Experimental :: @@ -192,7 +113,7 @@ def transform(self, vector): :return: Standardized vector. If the variance of a column is zero, it will return default `0.0` for the column with zero variance. """ - return JavaModelWrapper.transform(self, vector) + return JavaVectorTransformer.transform(self, vector) class StandardScaler(object): @@ -233,9 +154,8 @@ def fit(self, dataset): the transformation model. :return: a StandardScalarModel """ - sc = dataset.context - jmodel = _callAPI(sc, "fitStandardScaler", self.withMean, self.withStd, dataset) - return StandardScalerModel(sc, jmodel) + jmodel = callMLlibFunc("fitStandardScaler", self.withMean, self.withStd, dataset) + return StandardScalerModel(jmodel) class HashingTF(object): @@ -276,7 +196,7 @@ def transform(self, document): return Vectors.sparse(self.numFeatures, freq.items()) -class IDFModel(JavaModelWrapper): +class IDFModel(JavaVectorTransformer): """ Represents an IDF model that can transform term frequency vectors. """ @@ -291,7 +211,7 @@ def transform(self, dataset): :param dataset: an RDD of term frequency vectors :return: an RDD of TF-IDF vectors """ - return JavaModelWrapper.transform(self, dataset) + return JavaVectorTransformer.transform(self, dataset) class IDF(object): @@ -335,12 +255,11 @@ def fit(self, dataset): :param dataset: an RDD of term frequency vectors """ - sc = dataset.context - jmodel = _callAPI(sc, "fitIDF", self.minDocFreq, dataset) - return IDFModel(sc, jmodel) + jmodel = callMLlibFunc("fitIDF", self.minDocFreq, dataset) + return IDFModel(jmodel) -class Word2VecModel(JavaModelWrapper): +class Word2VecModel(JavaVectorTransformer): """ class for Word2Vec model """ @@ -354,7 +273,7 @@ def transform(self, word): :return: vector representation of word(s) """ try: - return _callJavaFunc(self._sc, self._java_model.transform, word) + return self.call("transform", word) except Py4JJavaError: raise ValueError("%s not found" % word) @@ -368,7 +287,7 @@ def findSynonyms(self, word, num): Note: local use only """ - words, similarity = _callJavaFunc(self._sc, self._java_model.findSynonyms, word, num) + words, similarity = self.call("findSynonyms", word, num) return zip(words, similarity) @@ -458,11 +377,10 @@ def fit(self, data): :param data: training data. RDD of subtype of Iterable[String] :return: Word2VecModel instance """ - sc = data.context - jmodel = _callAPI(sc, "trainWord2Vec", data, int(self.vectorSize), - float(self.learningRate), int(self.numPartitions), - int(self.numIterations), long(self.seed)) - return Word2VecModel(sc, jmodel) + jmodel = callMLlibFunc("trainWord2Vec", data, int(self.vectorSize), + float(self.learningRate), int(self.numPartitions), + int(self.numIterations), long(self.seed)) + return Word2VecModel(jmodel) def _test(): diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 1b9bf596242df..d0a0e102a1a07 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -29,7 +29,6 @@ import numpy as np -from pyspark.serializers import AutoBatchedSerializer, PickleSerializer __all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] @@ -52,17 +51,6 @@ def fast_pickle_array(ar): _have_scipy = False -# this will call the MLlib version of pythonToJava() -def _to_java_object_rdd(rdd): - """ Return an JavaRDD of Object by unpickling - - It will convert each Python object into Java object by Pyrolite, whenever the - RDD is serialized in batch or not. - """ - rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) - return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) - - def _convert_to_vector(l): if isinstance(l, Vector): return l diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 2202c51ab9c06..7eebfc6bcd894 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -21,22 +21,12 @@ from functools import wraps -from pyspark.rdd import RDD -from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.mllib.common import callMLlibFunc __all__ = ['RandomRDDs', ] -def serialize(f): - @wraps(f) - def func(sc, *a, **kw): - jrdd = f(sc, *a, **kw) - return RDD(sc._jvm.SerDe.javaToPython(jrdd), sc, - BatchedSerializer(PickleSerializer(), 1024)) - return func - - def toArray(f): @wraps(f) def func(sc, *a, **kw): @@ -52,7 +42,6 @@ class RandomRDDs(object): """ @staticmethod - @serialize def uniformRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the @@ -74,10 +63,9 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): >>> parts == sc.defaultParallelism True """ - return sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) + return callMLlibFunc("uniformRDD", sc._jsc, size, numPartitions, seed) @staticmethod - @serialize def normalRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the standard normal @@ -97,10 +85,9 @@ def normalRDD(sc, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - 1.0) < 0.1 True """ - return sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) + return callMLlibFunc("normalRDD", sc._jsc, size, numPartitions, seed) @staticmethod - @serialize def poissonRDD(sc, mean, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the Poisson @@ -117,11 +104,10 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - sqrt(mean)) < 0.5 True """ - return sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) + return callMLlibFunc("poissonRDD", sc._jsc, mean, size, numPartitions, seed) @staticmethod @toArray - @serialize def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -136,12 +122,10 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() 4 """ - return sc._jvm.PythonMLLibAPI() \ - .uniformVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) + return callMLlibFunc("uniformVectorRDD", sc._jsc, numRows, numCols, numPartitions, seed) @staticmethod @toArray - @serialize def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -156,12 +140,10 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - 1.0) < 0.1 True """ - return sc._jvm.PythonMLLibAPI() \ - .normalVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) + return callMLlibFunc("normalVectorRDD", sc._jsc, numRows, numCols, numPartitions, seed) @staticmethod @toArray - @serialize def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -179,8 +161,8 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - sqrt(mean)) < 0.5 True """ - return sc._jvm.PythonMLLibAPI() \ - .poissonVectorRDD(sc._jsc, mean, numRows, numCols, numPartitions, seed) + return callMLlibFunc("poissonVectorRDD", sc._jsc, mean, numRows, numCols, + numPartitions, seed) def _test(): diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 22872dbbe3b55..6b32af07c9be2 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -16,9 +16,8 @@ # from pyspark import SparkContext -from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.rdd import RDD -from pyspark.mllib.linalg import _to_java_object_rdd +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, _to_java_object_rdd __all__ = ['MatrixFactorizationModel', 'ALS'] @@ -36,7 +35,7 @@ def __repr__(self): return "Rating(%d, %d, %d)" % (self.user, self.product, self.rating) -class MatrixFactorizationModel(object): +class MatrixFactorizationModel(JavaModelWrapper): """A matrix factorisation model trained by regularized alternating least-squares. @@ -71,48 +70,21 @@ class MatrixFactorizationModel(object): >>> len(latents) == 4 True """ - - def __init__(self, sc, java_model): - self._context = sc - self._java_model = java_model - - def __del__(self): - self._context._gateway.detach(self._java_model) - def predict(self, user, product): return self._java_model.predict(user, product) def predictAll(self, user_product): assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" first = user_product.first() - if isinstance(first, list): - user_product = user_product.map(tuple) - first = tuple(first) - assert type(first) is tuple and len(first) == 2, \ - "user_product should be RDD of (user, product)" - if any(isinstance(x, str) for x in first): - user_product = user_product.map(lambda (u, p): (int(x), int(p))) - first = tuple(map(int, first)) - assert all(type(x) is int for x in first), "user and product in user_product shoul be int" - sc = self._context - tuplerdd = sc._jvm.SerDe.asTupleRDD(_to_java_object_rdd(user_product).rdd()) - jresult = self._java_model.predict(tuplerdd).toJavaRDD() - return RDD(sc._jvm.SerDe.javaToPython(jresult), sc, - AutoBatchedSerializer(PickleSerializer())) + assert len(first) == 2, "user_product should be RDD of (user, product)" + user_product = user_product.map(lambda (u, p): (int(u), int(p))) + return self.call("predict", user_product) def userFeatures(self): - sc = self._context - juf = self._java_model.userFeatures() - juf = sc._jvm.SerDe.fromTuple2RDD(juf).toJavaRDD() - return RDD(sc._jvm.PythonRDD.javaToPython(juf), sc, - AutoBatchedSerializer(PickleSerializer())) + return self.call("getUserFeatures") def productFeatures(self): - sc = self._context - jpf = self._java_model.productFeatures() - jpf = sc._jvm.SerDe.fromTuple2RDD(jpf).toJavaRDD() - return RDD(sc._jvm.PythonRDD.javaToPython(jpf), sc, - AutoBatchedSerializer(PickleSerializer())) + return self.call("getProductFeatures") class ALS(object): @@ -126,25 +98,19 @@ def _prepare(cls, ratings): ratings = ratings.map(lambda x: Rating(*x)) else: raise ValueError("rating should be RDD of Rating or tuple/list") - # serialize them by AutoBatchedSerializer before cache to reduce the - # objects overhead in JVM - cached = ratings._reserialize(AutoBatchedSerializer(PickleSerializer())).cache() - return _to_java_object_rdd(cached) + return _to_java_object_rdd(ratings, True) @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): - sc = ratings.context - jrating = cls._prepare(ratings) - mod = sc._jvm.PythonMLLibAPI().trainALSModel(jrating, rank, iterations, lambda_, blocks) - return MatrixFactorizationModel(sc, mod) + model = callMLlibFunc("trainALSModel", cls._prepare(ratings), rank, iterations, + lambda_, blocks) + return MatrixFactorizationModel(model) @classmethod def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01): - sc = ratings.context - jrating = cls._prepare(ratings) - mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel( - jrating, rank, iterations, lambda_, blocks, alpha) - return MatrixFactorizationModel(sc, mod) + model = callMLlibFunc("trainImplicitALSModel", cls._prepare(ratings), rank, + iterations, lambda_, blocks, alpha) + return MatrixFactorizationModel(model) def _test(): diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 93e17faf5cd51..43c1a2fc101dd 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -18,9 +18,8 @@ import numpy as np from numpy import array -from pyspark import SparkContext -from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, _to_java_object_rdd +from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] @@ -124,17 +123,11 @@ class LinearRegressionModel(LinearRegressionModelBase): # train_func should take two parameters, namely data and initial_weights, and # return the result of a call to the appropriate JVM stub. # _regression_train_wrapper is responsible for setup and error checking. -def _regression_train_wrapper(sc, train_func, modelClass, data, initial_weights): +def _regression_train_wrapper(train_func, modelClass, data, initial_weights): initial_weights = initial_weights or [0.0] * len(data.first().features) - ser = PickleSerializer() - initial_bytes = bytearray(ser.dumps(_convert_to_vector(initial_weights))) - # use AutoBatchedSerializer before cache to reduce the memory - # overhead in JVM - cached = data._reserialize(AutoBatchedSerializer(ser)).cache() - ans = train_func(_to_java_object_rdd(cached), initial_bytes) - assert len(ans) == 2, "JVM call result had unexpected length" - weights = ser.loads(str(ans[0])) - return modelClass(weights, ans[1]) + weights, intercept = train_func(_to_java_object_rdd(data, cache=True), + _convert_to_vector(initial_weights)) + return modelClass(weights, intercept) class LinearRegressionWithSGD(object): @@ -168,13 +161,12 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, training data (i.e. whether bias features are activated or not). """ - sc = data.context + def train(rdd, i): + return callMLlibFunc("trainLinearRegressionModelWithSGD", rdd, iterations, step, + miniBatchFraction, i, regParam, regType, intercept) - def train(jrdd, i): - return sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( - jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - - return _regression_train_wrapper(sc, train, LinearRegressionModel, data, initialWeights) + return _regression_train_wrapper(train, LinearRegressionModel, + data, initialWeights) class LassoModel(LinearRegressionModelBase): @@ -216,12 +208,10 @@ class LassoWithSGD(object): def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a Lasso regression model on the given data.""" - sc = data.context - - def train(jrdd, i): - return sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( - jrdd, iterations, step, regParam, miniBatchFraction, i) - return _regression_train_wrapper(sc, train, LassoModel, data, initialWeights) + def train(rdd, i): + return callMLlibFunc("trainLassoModelWithSGD", rdd, iterations, step, regParam, + miniBatchFraction, i) + return _regression_train_wrapper(train, LassoModel, data, initialWeights) class RidgeRegressionModel(LinearRegressionModelBase): @@ -263,17 +253,17 @@ class RidgeRegressionWithSGD(object): def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a ridge regression model on the given data.""" - sc = data.context - - def train(jrdd, i): - return sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( - jrdd, iterations, step, regParam, miniBatchFraction, i) + def train(rdd, i): + return callMLlibFunc("trainRidgeModelWithSGD", rdd, iterations, step, regParam, + miniBatchFraction, i) - return _regression_train_wrapper(sc, train, RidgeRegressionModel, data, initialWeights) + return _regression_train_wrapper(train, RidgeRegressionModel, + data, initialWeights) def _test(): import doctest + from pyspark import SparkContext globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 84baf12b906df..15f0652f833d7 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -19,66 +19,36 @@ Python package for statistical functions in MLlib. """ -from functools import wraps - -from pyspark import PickleSerializer -from pyspark.mllib.linalg import _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper +from pyspark.mllib.linalg import _convert_to_vector __all__ = ['MultivariateStatisticalSummary', 'Statistics'] -def serialize(f): - ser = PickleSerializer() - - @wraps(f) - def func(self): - jvec = f(self) - bytes = self._sc._jvm.SerDe.dumps(jvec) - return ser.loads(str(bytes)).toArray() - - return func - - -class MultivariateStatisticalSummary(object): +class MultivariateStatisticalSummary(JavaModelWrapper): """ Trait for multivariate statistical summary of a data matrix. """ - def __init__(self, sc, java_summary): - """ - :param sc: Spark context - :param java_summary: Handle to Java summary object - """ - self._sc = sc - self._java_summary = java_summary - - def __del__(self): - self._sc._gateway.detach(self._java_summary) - - @serialize def mean(self): - return self._java_summary.mean() + return self.call("mean").toArray() - @serialize def variance(self): - return self._java_summary.variance() + return self.call("variance").toArray() def count(self): - return self._java_summary.count() + return self.call("count") - @serialize def numNonzeros(self): - return self._java_summary.numNonzeros() + return self.call("numNonzeros").toArray() - @serialize def max(self): - return self._java_summary.max() + return self.call("max").toArray() - @serialize def min(self): - return self._java_summary.min() + return self.call("min").toArray() class Statistics(object): @@ -106,10 +76,8 @@ def colStats(rdd): >>> cStats.min() array([ 2., 0., 0., -2.]) """ - sc = rdd.ctx - jrdd = _to_java_object_rdd(rdd.map(_convert_to_vector)) - cStats = sc._jvm.PythonMLLibAPI().colStats(jrdd) - return MultivariateStatisticalSummary(sc, cStats) + cStats = callMLlibFunc("colStats", rdd.map(_convert_to_vector)) + return MultivariateStatisticalSummary(cStats) @staticmethod def corr(x, y=None, method=None): @@ -156,7 +124,6 @@ def corr(x, y=None, method=None): ... except TypeError: ... pass """ - sc = x.ctx # Check inputs to determine whether a single value or a matrix is needed for output. # Since it's legal for users to use the method name as the second argument, we need to # check if y is used to specify the method name instead. @@ -164,15 +131,9 @@ def corr(x, y=None, method=None): raise TypeError("Use 'method=' to specify method name.") if not y: - jx = _to_java_object_rdd(x.map(_convert_to_vector)) - resultMat = sc._jvm.PythonMLLibAPI().corr(jx, method) - bytes = sc._jvm.SerDe.dumps(resultMat) - ser = PickleSerializer() - return ser.loads(str(bytes)).toArray() + return callMLlibFunc("corr", x.map(_convert_to_vector), method).toArray() else: - jx = _to_java_object_rdd(x.map(float)) - jy = _to_java_object_rdd(y.map(float)) - return sc._jvm.PythonMLLibAPI().corr(jx, jy, method) + return callMLlibFunc("corr", x.map(float), y.map(float), method) def _test(): diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 64ee79d83e849..5d1a3c0962796 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -15,36 +15,22 @@ # limitations under the License. # -from py4j.java_collections import MapConverter - from pyspark import SparkContext, RDD -from pyspark.serializers import BatchedSerializer, PickleSerializer -from pyspark.mllib.linalg import Vector, _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper +from pyspark.mllib.linalg import _convert_to_vector from pyspark.mllib.regression import LabeledPoint __all__ = ['DecisionTreeModel', 'DecisionTree'] -class DecisionTreeModel(object): +class DecisionTreeModel(JavaModelWrapper): """ A decision tree model for classification or regression. EXPERIMENTAL: This is an experimental API. - It will probably be modified for Spark v1.2. + It will probably be modified in future. """ - - def __init__(self, sc, java_model): - """ - :param sc: Spark context - :param java_model: Handle to Java model object - """ - self._sc = sc - self._java_model = java_model - - def __del__(self): - self._sc._gateway.detach(self._java_model) - def predict(self, x): """ Predict the label of one or more examples. @@ -52,24 +38,11 @@ def predict(self, x): :param x: Data point (feature vector), or an RDD of data points (feature vectors). """ - SerDe = self._sc._jvm.SerDe - ser = PickleSerializer() if isinstance(x, RDD): - # Bulk prediction - first = x.take(1) - if not first: - return self._sc.parallelize([]) - if not isinstance(first[0], Vector): - x = x.map(_convert_to_vector) - jPred = self._java_model.predict(_to_java_object_rdd(x)).toJavaRDD() - jpyrdd = self._sc._jvm.SerDe.javaToPython(jPred) - return RDD(jpyrdd, self._sc, BatchedSerializer(ser, 1024)) + return self.call("predict", x.map(_convert_to_vector)) else: - # Assume x is a single data point. - bytes = bytearray(ser.dumps(_convert_to_vector(x))) - vec = self._sc._jvm.SerDe.loads(bytes) - return self._java_model.predict(vec) + return self.call("predict", _convert_to_vector(x)) def numNodes(self): return self._java_model.numNodes() @@ -98,19 +71,13 @@ class DecisionTree(object): """ @staticmethod - def _train(data, type, numClasses, categoricalFeaturesInfo, - impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, - minInfoGain=0.0): + def _train(data, type, numClasses, features, impurity="gini", maxDepth=5, maxBins=32, + minInstancesPerNode=1, minInfoGain=0.0): first = data.first() assert isinstance(first, LabeledPoint), "the data should be RDD of LabeledPoint" - sc = data.context - jrdd = _to_java_object_rdd(data) - cfiMap = MapConverter().convert(categoricalFeaturesInfo, - sc._gateway._gateway_client) - model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - jrdd, type, numClasses, cfiMap, - impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) - return DecisionTreeModel(sc, model) + model = callMLlibFunc("trainDecisionTreeModel", data, type, numClasses, features, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) + return DecisionTreeModel(model) @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 84b39a48619d2..96aef8f510fa6 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -18,8 +18,7 @@ import numpy as np import warnings -from pyspark.rdd import RDD -from pyspark.serializers import AutoBatchedSerializer, PickleSerializer +from pyspark.mllib.common import callMLlibFunc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -173,9 +172,7 @@ def loadLabeledPoints(sc, path, minPartitions=None): (0.0,[1.01,2.02,3.03]) """ minPartitions = minPartitions or min(sc.defaultParallelism, 2) - jrdd = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) - jpyrdd = sc._jvm.SerDe.javaToPython(jrdd) - return RDD(jpyrdd, sc, AutoBatchedSerializer(PickleSerializer())) + return callMLlibFunc("loadLabeledPoints", sc, path, minPartitions) def _test(): From ad3bd0dff8997861c5a04438145ba6f91c57a849 Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Thu, 30 Oct 2014 22:30:52 -0700 Subject: [PATCH 102/115] [SPARK-3250] Implement Gap Sampling optimization for random sampling More efficient sampling, based on Gap Sampling optimization: http://erikerlandson.github.io/blog/2014/09/11/faster-random-samples-with-gap-sampling/ Author: Erik Erlandson Closes #2455 from erikerlandson/spark-3250-pr and squashes the following commits: 72496bc [Erik Erlandson] [SPARK-3250] Implement Gap Sampling optimization for random sampling --- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../spark/util/random/RandomSampler.scala | 286 ++++++++- .../java/org/apache/spark/JavaAPISuite.java | 9 +- .../util/random/RandomSamplerSuite.scala | 606 +++++++++++++++--- .../org/apache/spark/mllib/util/MLUtils.scala | 4 +- 5 files changed, 790 insertions(+), 121 deletions(-) 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 b7f125d01dfaf..c169b2d3fe97f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -43,7 +43,8 @@ import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{BoundedPriorityQueue, Utils, CallSite} import org.apache.spark.util.collection.OpenHashMap -import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} +import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, BernoulliCellSampler, + SamplingUtils} /** * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, @@ -375,7 +376,8 @@ abstract class RDD[T: ClassTag]( val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => - new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](x(0), x(1)), true, seed) + new PartitionwiseSampledRDD[T, T]( + this, new BernoulliCellSampler[T](x(0), x(1)), true, seed) }.toArray } diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index ee389def20c8c..76e7a2760bcd1 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -19,6 +19,9 @@ package org.apache.spark.util.random import java.util.Random +import scala.reflect.ClassTag +import scala.collection.mutable.ArrayBuffer + import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.annotation.DeveloperApi @@ -38,13 +41,47 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable /** take a random sample */ def sample(items: Iterator[T]): Iterator[U] + /** return a copy of the RandomSampler object */ override def clone: RandomSampler[T, U] = throw new NotImplementedError("clone() is not implemented.") } +private[spark] +object RandomSampler { + /** Default random number generator used by random samplers. */ + def newDefaultRNG: Random = new XORShiftRandom + + /** + * Default maximum gap-sampling fraction. + * For sampling fractions <= this value, the gap sampling optimization will be applied. + * Above this value, it is assumed that "tradtional" Bernoulli sampling is faster. The + * optimal value for this will depend on the RNG. More expensive RNGs will tend to make + * the optimal value higher. The most reliable way to determine this value for a new RNG + * is to experiment. When tuning for a new RNG, I would expect a value of 0.5 to be close + * in most cases, as an initial guess. + */ + val defaultMaxGapSamplingFraction = 0.4 + + /** + * Default epsilon for floating point numbers sampled from the RNG. + * The gap-sampling compute logic requires taking log(x), where x is sampled from an RNG. + * To guard against errors from taking log(0), a positive epsilon lower bound is applied. + * A good value for this parameter is at or near the minimum positive floating + * point value returned by "nextDouble()" (or equivalent), for the RNG being used. + */ + val rngEpsilon = 5e-11 + + /** + * Sampling fraction arguments may be results of computation, and subject to floating + * point jitter. I check the arguments with this epsilon slop factor to prevent spurious + * warnings for cases such as summing some numbers to get a sampling fraction of 1.000000001 + */ + val roundingEpsilon = 1e-6 +} + /** * :: DeveloperApi :: - * A sampler based on Bernoulli trials. + * A sampler based on Bernoulli trials for partitioning a data sequence. * * @param lb lower bound of the acceptance range * @param ub upper bound of the acceptance range @@ -52,57 +89,262 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @tparam T item type */ @DeveloperApi -class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) +class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = false) extends RandomSampler[T, T] { - private[random] var rng: Random = new XORShiftRandom + /** epsilon slop to avoid failure from floating point jitter. */ + require( + lb <= (ub + RandomSampler.roundingEpsilon), + s"Lower bound ($lb) must be <= upper bound ($ub)") + require( + lb >= (0.0 - RandomSampler.roundingEpsilon), + s"Lower bound ($lb) must be >= 0.0") + require( + ub <= (1.0 + RandomSampler.roundingEpsilon), + s"Upper bound ($ub) must be <= 1.0") - def this(ratio: Double) = this(0.0d, ratio) + private val rng: Random = new XORShiftRandom override def setSeed(seed: Long) = rng.setSeed(seed) override def sample(items: Iterator[T]): Iterator[T] = { - items.filter { item => - val x = rng.nextDouble() - (x >= lb && x < ub) ^ complement + if (ub - lb <= 0.0) { + if (complement) items else Iterator.empty + } else { + if (complement) { + items.filter { item => { + val x = rng.nextDouble() + (x < lb) || (x >= ub) + }} + } else { + items.filter { item => { + val x = rng.nextDouble() + (x >= lb) && (x < ub) + }} + } } } /** * Return a sampler that is the complement of the range specified of the current sampler. */ - def cloneComplement(): BernoulliSampler[T] = new BernoulliSampler[T](lb, ub, !complement) + def cloneComplement(): BernoulliCellSampler[T] = + new BernoulliCellSampler[T](lb, ub, !complement) + + override def clone = new BernoulliCellSampler[T](lb, ub, complement) +} + + +/** + * :: DeveloperApi :: + * A sampler based on Bernoulli trials. + * + * @param fraction the sampling fraction, aka Bernoulli sampling probability + * @tparam T item type + */ +@DeveloperApi +class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T] { + + /** epsilon slop to avoid failure from floating point jitter */ + require( + fraction >= (0.0 - RandomSampler.roundingEpsilon) + && fraction <= (1.0 + RandomSampler.roundingEpsilon), + s"Sampling fraction ($fraction) must be on interval [0, 1]") - override def clone = new BernoulliSampler[T](lb, ub, complement) + private val rng: Random = RandomSampler.newDefaultRNG + + override def setSeed(seed: Long) = rng.setSeed(seed) + + override def sample(items: Iterator[T]): Iterator[T] = { + if (fraction <= 0.0) { + Iterator.empty + } else if (fraction >= 1.0) { + items + } else if (fraction <= RandomSampler.defaultMaxGapSamplingFraction) { + new GapSamplingIterator(items, fraction, rng, RandomSampler.rngEpsilon) + } else { + items.filter { _ => rng.nextDouble() <= fraction } + } + } + + override def clone = new BernoulliSampler[T](fraction) } + /** * :: DeveloperApi :: - * A sampler based on values drawn from Poisson distribution. + * A sampler for sampling with replacement, based on values drawn from Poisson distribution. * - * @param mean Poisson mean + * @param fraction the sampling fraction (with replacement) * @tparam T item type */ @DeveloperApi -class PoissonSampler[T](mean: Double) extends RandomSampler[T, T] { +class PoissonSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T] { + + /** Epsilon slop to avoid failure from floating point jitter. */ + require( + fraction >= (0.0 - RandomSampler.roundingEpsilon), + s"Sampling fraction ($fraction) must be >= 0") - private[random] var rng = new PoissonDistribution(mean) + // PoissonDistribution throws an exception when fraction <= 0 + // If fraction is <= 0, Iterator.empty is used below, so we can use any placeholder value. + private val rng = new PoissonDistribution(if (fraction > 0.0) fraction else 1.0) + private val rngGap = RandomSampler.newDefaultRNG override def setSeed(seed: Long) { - rng = new PoissonDistribution(mean) rng.reseedRandomGenerator(seed) + rngGap.setSeed(seed) } override def sample(items: Iterator[T]): Iterator[T] = { - items.flatMap { item => - val count = rng.sample() - if (count == 0) { - Iterator.empty - } else { - Iterator.fill(count)(item) - } + if (fraction <= 0.0) { + Iterator.empty + } else if (fraction <= RandomSampler.defaultMaxGapSamplingFraction) { + new GapSamplingReplacementIterator(items, fraction, rngGap, RandomSampler.rngEpsilon) + } else { + items.flatMap { item => { + val count = rng.sample() + if (count == 0) Iterator.empty else Iterator.fill(count)(item) + }} + } + } + + override def clone = new PoissonSampler[T](fraction) +} + + +private[spark] +class GapSamplingIterator[T: ClassTag]( + var data: Iterator[T], + f: Double, + rng: Random = RandomSampler.newDefaultRNG, + epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] { + + require(f > 0.0 && f < 1.0, s"Sampling fraction ($f) must reside on open interval (0, 1)") + require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0") + + /** implement efficient linear-sequence drop until Scala includes fix for jira SI-8835. */ + private val iterDrop: Int => Unit = { + val arrayClass = Array.empty[T].iterator.getClass + val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass + data.getClass match { + case `arrayClass` => ((n: Int) => { data = data.drop(n) }) + case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) }) + case _ => ((n: Int) => { + var j = 0 + while (j < n && data.hasNext) { + data.next() + j += 1 + } + }) + } + } + + override def hasNext: Boolean = data.hasNext + + override def next(): T = { + val r = data.next() + advance + r + } + + private val lnq = math.log1p(-f) + + /** skip elements that won't be sampled, according to geometric dist P(k) = (f)(1-f)^k. */ + private def advance: Unit = { + val u = math.max(rng.nextDouble(), epsilon) + val k = (math.log(u) / lnq).toInt + iterDrop(k) + } + + /** advance to first sample as part of object construction. */ + advance + // Attempting to invoke this closer to the top with other object initialization + // was causing it to break in strange ways, so I'm invoking it last, which seems to + // work reliably. +} + +private[spark] +class GapSamplingReplacementIterator[T: ClassTag]( + var data: Iterator[T], + f: Double, + rng: Random = RandomSampler.newDefaultRNG, + epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] { + + require(f > 0.0, s"Sampling fraction ($f) must be > 0") + require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0") + + /** implement efficient linear-sequence drop until scala includes fix for jira SI-8835. */ + private val iterDrop: Int => Unit = { + val arrayClass = Array.empty[T].iterator.getClass + val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass + data.getClass match { + case `arrayClass` => ((n: Int) => { data = data.drop(n) }) + case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) }) + case _ => ((n: Int) => { + var j = 0 + while (j < n && data.hasNext) { + data.next() + j += 1 + } + }) + } + } + + /** current sampling value, and its replication factor, as we are sampling with replacement. */ + private var v: T = _ + private var rep: Int = 0 + + override def hasNext: Boolean = data.hasNext || rep > 0 + + override def next(): T = { + val r = v + rep -= 1 + if (rep <= 0) advance + r + } + + /** + * Skip elements with replication factor zero (i.e. elements that won't be sampled). + * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is + * q is the probabililty of Poisson(0; f) + */ + private def advance: Unit = { + val u = math.max(rng.nextDouble(), epsilon) + val k = (math.log(u) / (-f)).toInt + iterDrop(k) + // set the value and replication factor for the next value + if (data.hasNext) { + v = data.next() + rep = poissonGE1 + } + } + + private val q = math.exp(-f) + + /** + * Sample from Poisson distribution, conditioned such that the sampled value is >= 1. + * This is an adaptation from the algorithm for Generating Poisson distributed random variables: + * http://en.wikipedia.org/wiki/Poisson_distribution + */ + private def poissonGE1: Int = { + // simulate that the standard poisson sampling + // gave us at least one iteration, for a sample of >= 1 + var pp = q + ((1.0 - q) * rng.nextDouble()) + var r = 1 + + // now continue with standard poisson sampling algorithm + pp *= rng.nextDouble() + while (pp > q) { + r += 1 + pp *= rng.nextDouble() } + r } - override def clone = new PoissonSampler[T](mean) + /** advance to first sample as part of object construction. */ + advance + // Attempting to invoke this closer to the top with other object initialization + // was causing it to break in strange ways, so I'm invoking it last, which seems to + // work reliably. } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 0172876a264b8..c21a4b30d7726 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -140,11 +140,10 @@ public void intersection() { public void sample() { List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); JavaRDD rdd = sc.parallelize(ints); - JavaRDD sample20 = rdd.sample(true, 0.2, 11); - // expected 2 but of course result varies randomly a bit - Assert.assertEquals(1, sample20.count()); - JavaRDD sample20NoReplacement = rdd.sample(false, 0.2, 11); - Assert.assertEquals(2, sample20NoReplacement.count()); + JavaRDD sample20 = rdd.sample(true, 0.2, 3); + Assert.assertEquals(2, sample20.count()); + JavaRDD sample20WithoutReplacement = rdd.sample(false, 0.2, 5); + Assert.assertEquals(2, sample20WithoutReplacement.count()); } @Test diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index ba67d766a775a..20944b62473c5 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -18,97 +18,523 @@ package org.apache.spark.util.random import java.util.Random - +import scala.collection.mutable.ArrayBuffer import org.apache.commons.math3.distribution.PoissonDistribution -import org.scalatest.{BeforeAndAfter, FunSuite} -import org.scalatest.mock.EasyMockSugar - -class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { - - val a = List(1, 2, 3, 4, 5, 6, 7, 8, 9) - - var random: Random = _ - var poisson: PoissonDistribution = _ - - before { - random = mock[Random] - poisson = mock[PoissonDistribution] - } - - test("BernoulliSamplerWithRange") { - expecting { - for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { - random.nextDouble().andReturn(x) - } - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55) - sampler.rng = random - assert(sampler.sample(a.iterator).toList == List(3, 4, 5)) - } - } - - test("BernoulliSamplerWithRangeInverse") { - expecting { - for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { - random.nextDouble().andReturn(x) - } - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55, true) - sampler.rng = random - assert(sampler.sample(a.iterator).toList === List(1, 2, 6, 7, 8, 9)) - } - } - - test("BernoulliSamplerWithRatio") { - expecting { - for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { - random.nextDouble().andReturn(x) - } - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.35) - sampler.rng = random - assert(sampler.sample(a.iterator).toList == List(1, 2, 3)) - } - } - - test("BernoulliSamplerWithComplement") { - expecting { - for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { - random.nextDouble().andReturn(x) - } - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55, true) - sampler.rng = random - assert(sampler.sample(a.iterator).toList == List(1, 2, 6, 7, 8, 9)) - } - } - - test("BernoulliSamplerSetSeed") { - expecting { - random.setSeed(10L) - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.2) - sampler.rng = random - sampler.setSeed(10L) - } - } - - test("PoissonSampler") { - expecting { - for(x <- Seq(0, 1, 2, 0, 1, 1, 0, 0, 0)) { - poisson.sample().andReturn(x) - } - } - whenExecuting(poisson) { - val sampler = new PoissonSampler[Int](0.2) - sampler.rng = poisson - assert(sampler.sample(a.iterator).toList == List(2, 3, 3, 5, 6)) - } +import org.scalatest.{FunSuite, Matchers} + +class RandomSamplerSuite extends FunSuite with Matchers { + /** + * My statistical testing methodology is to run a Kolmogorov-Smirnov (KS) test + * between the random samplers and simple reference samplers (known to work correctly). + * The sampling gap sizes between chosen samples should show up as having the same + * distributions between test and reference, if things are working properly. That is, + * the KS test will fail to strongly reject the null hypothesis that the distributions of + * sampling gaps are the same. + * There are no actual KS tests implemented for scala (that I can find) - and so what I + * have done here is pre-compute "D" - the KS statistic - that corresponds to a "weak" + * p-value for a particular sample size. I can then test that my measured KS stats + * are less than D. Computing D-values is easy, and implemented below. + * + * I used the scipy 'kstwobign' distribution to pre-compute my D value: + * + * def ksdval(q=0.1, n=1000): + * en = np.sqrt(float(n) / 2.0) + * return stats.kstwobign.isf(float(q)) / (en + 0.12 + 0.11 / en) + * + * When comparing KS stats I take the median of a small number of independent test runs + * to compensate for the issue that any sampled statistic will show "false positive" with + * some probability. Even when two distributions are the same, they will register as + * different 10% of the time at a p-value of 0.1 + */ + + // This D value is the precomputed KS statistic for p-value 0.1, sample size 1000: + val sampleSize = 1000 + val D = 0.0544280747619 + + // I'm not a big fan of fixing seeds, but unit testing based on running statistical tests + // will always fail with some nonzero probability, so I'll fix the seed to prevent these + // tests from generating random failure noise in CI testing, etc. + val rngSeed: Random = RandomSampler.newDefaultRNG + rngSeed.setSeed(235711) + + // Reference implementation of sampling without replacement (bernoulli) + def sample[T](data: Iterator[T], f: Double): Iterator[T] = { + val rng: Random = RandomSampler.newDefaultRNG + rng.setSeed(rngSeed.nextLong) + data.filter(_ => (rng.nextDouble <= f)) + } + + // Reference implementation of sampling with replacement + def sampleWR[T](data: Iterator[T], f: Double): Iterator[T] = { + val rng = new PoissonDistribution(f) + rng.reseedRandomGenerator(rngSeed.nextLong) + data.flatMap { v => { + val rep = rng.sample() + if (rep == 0) Iterator.empty else Iterator.fill(rep)(v) + }} + } + + // Returns iterator over gap lengths between samples. + // This function assumes input data is integers sampled from the sequence of + // increasing integers: {0, 1, 2, ...}. This works because that is how I generate them, + // and the samplers preserve their input order + def gaps(data: Iterator[Int]): Iterator[Int] = { + data.sliding(2).withPartial(false).map { x => x(1) - x(0) } + } + + // Returns the cumulative distribution from a histogram + def cumulativeDist(hist: Array[Int]): Array[Double] = { + val n = hist.sum.toDouble + assert(n > 0.0) + hist.scanLeft(0)(_ + _).drop(1).map { _.toDouble / n } + } + + // Returns aligned cumulative distributions from two arrays of data + def cumulants(d1: Array[Int], d2: Array[Int], + ss: Int = sampleSize): (Array[Double], Array[Double]) = { + assert(math.min(d1.length, d2.length) > 0) + assert(math.min(d1.min, d2.min) >= 0) + val m = 1 + math.max(d1.max, d2.max) + val h1 = Array.fill[Int](m)(0) + val h2 = Array.fill[Int](m)(0) + for (v <- d1) { h1(v) += 1 } + for (v <- d2) { h2(v) += 1 } + assert(h1.sum == h2.sum) + assert(h1.sum == ss) + (cumulativeDist(h1), cumulativeDist(h2)) + } + + // Computes the Kolmogorov-Smirnov 'D' statistic from two cumulative distributions + def KSD(cdf1: Array[Double], cdf2: Array[Double]): Double = { + assert(cdf1.length == cdf2.length) + val n = cdf1.length + assert(n > 0) + assert(cdf1(n-1) == 1.0) + assert(cdf2(n-1) == 1.0) + cdf1.zip(cdf2).map { x => Math.abs(x._1 - x._2) }.max + } + + // Returns the median KS 'D' statistic between two samples, over (m) sampling trials + def medianKSD(data1: => Iterator[Int], data2: => Iterator[Int], m: Int = 5): Double = { + val t = Array.fill[Double](m) { + val (c1, c2) = cumulants(data1.take(sampleSize).toArray, + data2.take(sampleSize).toArray) + KSD(c1, c2) + }.sorted + // return the median KS statistic + t(m / 2) + } + + test("utilities") { + val s1 = Array(0, 1, 1, 0, 2) + val s2 = Array(1, 0, 3, 2, 1) + val (c1, c2) = cumulants(s1, s2, ss = 5) + c1 should be (Array(0.4, 0.8, 1.0, 1.0)) + c2 should be (Array(0.2, 0.6, 0.8, 1.0)) + KSD(c1, c2) should be (0.2 +- 0.000001) + KSD(c2, c1) should be (KSD(c1, c2)) + gaps(List(0, 1, 1, 2, 4, 11).iterator).toArray should be (Array(1, 0, 1, 2, 7)) + } + + test("sanity check medianKSD against references") { + var d: Double = 0.0 + + // should be statistically same, i.e. fail to reject null hypothesis strongly + d = medianKSD(gaps(sample(Iterator.from(0), 0.5)), gaps(sample(Iterator.from(0), 0.5))) + d should be < D + + // should be statistically different - null hypothesis will have high D value, + // corresponding to low p-value that rejects the null hypothesis + d = medianKSD(gaps(sample(Iterator.from(0), 0.4)), gaps(sample(Iterator.from(0), 0.5))) + d should be > D + + // same! + d = medianKSD(gaps(sampleWR(Iterator.from(0), 0.5)), gaps(sampleWR(Iterator.from(0), 0.5))) + d should be < D + + // different! + d = medianKSD(gaps(sampleWR(Iterator.from(0), 0.5)), gaps(sampleWR(Iterator.from(0), 0.6))) + d should be > D + } + + test("bernoulli sampling") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + var sampler: RandomSampler[Int, Int] = new BernoulliSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.5))) + d should be < D + + sampler = new BernoulliSampler[Int](0.7) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.7))) + d should be < D + + sampler = new BernoulliSampler[Int](0.9) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.9))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new BernoulliSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.6))) + d should be > D + } + + test("bernoulli sampling with gap sampling optimization") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + var sampler: RandomSampler[Int, Int] = new BernoulliSampler[Int](0.01) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.01))) + d should be < D + + sampler = new BernoulliSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + sampler = new BernoulliSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.3))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new BernoulliSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.4))) + d should be > D + } + + test("bernoulli boundary cases") { + val data = (1 to 100).toArray + + var sampler = new BernoulliSampler[Int](0.0) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliSampler[Int](1.0) + sampler.sample(data.iterator).toArray should be (data) + + sampler = new BernoulliSampler[Int](0.0 - (RandomSampler.roundingEpsilon / 2.0)) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliSampler[Int](1.0 + (RandomSampler.roundingEpsilon / 2.0)) + sampler.sample(data.iterator).toArray should be (data) + } + + test("bernoulli data types") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + var sampler = new BernoulliSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + + // Array iterator (indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).toArray.iterator)), + gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + // ArrayBuffer iterator (indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).to[ArrayBuffer].iterator)), + gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + // List iterator (non-indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).toList.iterator)), + gaps(sample(Iterator.from(0), 0.1))) + d should be < D + } + + test("bernoulli clone") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d = 0.0 + var sampler = new BernoulliSampler[Int](0.1).clone + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + sampler = new BernoulliSampler[Int](0.9).clone + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.9))) + d should be < D + } + + test("bernoulli set seed") { + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + var sampler1 = new BernoulliSampler[Int](0.2) + var sampler2 = new BernoulliSampler[Int](0.2) + + // distributions should be identical if seeds are set same + sampler1.setSeed(73) + sampler2.setSeed(73) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be (0.0) + + // should be different for different seeds + sampler1.setSeed(73) + sampler2.setSeed(37) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be > 0.0 + d should be < D + + sampler1 = new BernoulliSampler[Int](0.8) + sampler2 = new BernoulliSampler[Int](0.8) + + // distributions should be identical if seeds are set same + sampler1.setSeed(73) + sampler2.setSeed(73) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be (0.0) + + // should be different for different seeds + sampler1.setSeed(73) + sampler2.setSeed(37) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be > 0.0 + d should be < D + } + + test("replacement sampling") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + var sampler = new PoissonSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.5))) + d should be < D + + sampler = new PoissonSampler[Int](0.7) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.7))) + d should be < D + + sampler = new PoissonSampler[Int](0.9) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.9))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new PoissonSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.6))) + d should be > D + } + + test("replacement sampling with gap sampling") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + var sampler = new PoissonSampler[Int](0.01) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.01))) + d should be < D + + sampler = new PoissonSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + sampler = new PoissonSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.3))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new PoissonSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.4))) + d should be > D + } + + test("replacement boundary cases") { + val data = (1 to 100).toArray + + var sampler = new PoissonSampler[Int](0.0) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new PoissonSampler[Int](0.0 - (RandomSampler.roundingEpsilon / 2.0)) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + // sampling with replacement has no upper bound on sampling fraction + sampler = new PoissonSampler[Int](2.0) + sampler.sample(data.iterator).length should be > (data.length) + } + + test("replacement data types") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + var sampler = new PoissonSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + + // Array iterator (indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).toArray.iterator)), + gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + // ArrayBuffer iterator (indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).to[ArrayBuffer].iterator)), + gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + // List iterator (non-indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).toList.iterator)), + gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + } + + test("replacement clone") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d = 0.0 + var sampler = new PoissonSampler[Int](0.1).clone + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + sampler = new PoissonSampler[Int](0.9).clone + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.9))) + d should be < D + } + + test("replacement set seed") { + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + var sampler1 = new PoissonSampler[Int](0.2) + var sampler2 = new PoissonSampler[Int](0.2) + + // distributions should be identical if seeds are set same + sampler1.setSeed(73) + sampler2.setSeed(73) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be (0.0) + + // should be different for different seeds + sampler1.setSeed(73) + sampler2.setSeed(37) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be > 0.0 + d should be < D + + sampler1 = new PoissonSampler[Int](0.8) + sampler2 = new PoissonSampler[Int](0.8) + + // distributions should be identical if seeds are set same + sampler1.setSeed(73) + sampler2.setSeed(73) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be (0.0) + + // should be different for different seeds + sampler1.setSeed(73) + sampler2.setSeed(37) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be > 0.0 + d should be < D + } + + test("bernoulli partitioning sampling") { + var d: Double = 0.0 + + var sampler = new BernoulliCellSampler[Int](0.1, 0.2) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + sampler = new BernoulliCellSampler[Int](0.1, 0.2, true) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.9))) + d should be < D + } + + test("bernoulli partitioning boundary cases") { + val data = (1 to 100).toArray + val d = RandomSampler.roundingEpsilon / 2.0 + + var sampler = new BernoulliCellSampler[Int](0.0, 0.0) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](1.0, 1.0) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](0.0, 1.0) + sampler.sample(data.iterator).toArray should be (data) + + sampler = new BernoulliCellSampler[Int](0.0 - d, 1.0 + d) + sampler.sample(data.iterator).toArray should be (data) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5 - d) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + } + + test("bernoulli partitioning data") { + val seed = rngSeed.nextLong + val data = (1 to 100).toArray + + var sampler = new BernoulliCellSampler[Int](0.4, 0.6) + sampler.setSeed(seed) + val s1 = sampler.sample(data.iterator).toArray + s1.length should be > 0 + + sampler = new BernoulliCellSampler[Int](0.4, 0.6, true) + sampler.setSeed(seed) + val s2 = sampler.sample(data.iterator).toArray + s2.length should be > 0 + + (s1 ++ s2).sorted should be (data) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5, true) + sampler.sample(data.iterator).toArray should be (data) + } + + test("bernoulli partitioning clone") { + val seed = rngSeed.nextLong + val data = (1 to 100).toArray + val base = new BernoulliCellSampler[Int](0.35, 0.65) + + var sampler = base.clone + sampler.setSeed(seed) + val s1 = sampler.sample(data.iterator).toArray + s1.length should be > 0 + + sampler = base.cloneComplement + sampler.setSeed(seed) + val s2 = sampler.sample(data.iterator).toArray + s2.length should be > 0 + + (s1 ++ s2).sorted should be (data) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index b88e08bf148ae..9353351af72a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -26,7 +26,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD -import org.apache.spark.util.random.BernoulliSampler +import org.apache.spark.util.random.BernoulliCellSampler import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.storage.StorageLevel @@ -244,7 +244,7 @@ object MLUtils { def kFold[T: ClassTag](rdd: RDD[T], numFolds: Int, seed: Int): Array[(RDD[T], RDD[T])] = { val numFoldsF = numFolds.toFloat (1 to numFolds).map { fold => - val sampler = new BernoulliSampler[T]((fold - 1) / numFoldsF, fold / numFoldsF, + val sampler = new BernoulliCellSampler[T]((fold - 1) / numFoldsF, fold / numFoldsF, complement = false) val validation = new PartitionwiseSampledRDD(rdd, sampler, true, seed) val training = new PartitionwiseSampledRDD(rdd, sampler.cloneComplement(), true, seed) From d31517a3cd6f887cc66ffd3c8e0c12bace3a948d Mon Sep 17 00:00:00 2001 From: Anant Date: Thu, 30 Oct 2014 23:02:42 -0700 Subject: [PATCH 103/115] [SPARK-4108][SQL] Fixed usage of deprecated in sql/catalyst/types/datatypes Fixed usage of deprecated in sql/catalyst/types/datatypes to have versio...n parameter Author: Anant Closes #2970 from anantasty/SPARK-4108 and squashes the following commits: e92cb01 [Anant] Fixed usage of deprecated in sql/catalyst/types/datatypes to have version parameter --- .../scala/org/apache/spark/sql/catalyst/types/dataTypes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index b9cf37d53ffd2..4e6e1166bfffb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -76,7 +76,7 @@ object DataType { StructField(name, parseDataType(dataType), nullable) } - @deprecated("Use DataType.fromJson instead") + @deprecated("Use DataType.fromJson instead", "1.2.0") def fromCaseClassString(string: String): DataType = CaseClassStringParser(string) private object CaseClassStringParser extends RegexParsers { From 58a6077e56f0f8f75fdd57d3aaad7485aeb07615 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 30 Oct 2014 23:59:46 -0700 Subject: [PATCH 104/115] [SPARK-4143] [SQL] Move inner class DeferredObjectAdapter to top level The class DeferredObjectAdapter is the inner class of HiveGenericUdf, which may cause some overhead in closure ser/de-ser. Move it to top level. Author: Cheng Hao Closes #3007 from chenghao-intel/move_deferred and squashes the following commits: 3a139b1 [Cheng Hao] Move inner class DeferredObjectAdapter to top level --- .../org/apache/spark/sql/hive/hiveUdfs.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index aff4ddce92272..86f7eea5dfd69 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis @@ -134,11 +135,19 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ } } +// Adapter from Catalyst ExpressionResult to Hive DeferredObject +private[hive] class DeferredObjectAdapter(oi: ObjectInspector) + extends DeferredObject with HiveInspectors { + private var func: () => Any = _ + def set(func: () => Any) { + this.func = func + } + override def prepare(i: Int) = {} + override def get(): AnyRef = wrap(func(), oi) +} + private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq[Expression]) extends HiveUdf with HiveInspectors { - - import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ - type UDFType = GenericUDF @transient @@ -161,16 +170,6 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq protected lazy val deferedObjects = argumentInspectors.map(new DeferredObjectAdapter(_)).toArray[DeferredObject] - // Adapter from Catalyst ExpressionResult to Hive DeferredObject - class DeferredObjectAdapter(oi: ObjectInspector) extends DeferredObject { - private var func: () => Any = _ - def set(func: () => Any) { - this.func = func - } - override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(func(), oi) - } - lazy val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: Row): Any = { From acd4ac7c9a503445e27739708cf36e19119b8ddc Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 31 Oct 2014 08:43:06 -0500 Subject: [PATCH 105/115] SPARK-3837. Warn when YARN kills containers for exceeding memory limits I triggered the issue and verified the message gets printed on a pseudo-distributed cluster. Author: Sandy Ryza Closes #2744 from sryza/sandy-spark-3837 and squashes the following commits: 858a268 [Sandy Ryza] Review feedback c937f00 [Sandy Ryza] SPARK-3837. Warn when YARN kills containers for exceeding memory limits --- .../spark/deploy/yarn/YarnAllocator.scala | 30 ++++++++++++++-- .../deploy/yarn/YarnAllocatorSuite.scala | 34 +++++++++++++++++++ 2 files changed, 61 insertions(+), 3 deletions(-) create mode 100644 yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 7ae8ef237ff89..e6196194acbb4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.yarn import java.util.{List => JList} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger +import java.util.regex.Pattern import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -375,12 +376,22 @@ private[yarn] abstract class YarnAllocator( logInfo("Completed container %s (state: %s, exit status: %s)".format( containerId, completedContainer.getState, - completedContainer.getExitStatus())) + completedContainer.getExitStatus)) // Hadoop 2.2.X added a ContainerExitStatus we should switch to use // there are some exit status' we shouldn't necessarily count against us, but for // now I think its ok as none of the containers are expected to exit - if (completedContainer.getExitStatus() != 0) { - logInfo("Container marked as failed: " + containerId) + if (completedContainer.getExitStatus == -103) { // vmem limit exceeded + logWarning(memLimitExceededLogMessage( + completedContainer.getDiagnostics, + VMEM_EXCEEDED_PATTERN)) + } else if (completedContainer.getExitStatus == -104) { // pmem limit exceeded + logWarning(memLimitExceededLogMessage( + completedContainer.getDiagnostics, + PMEM_EXCEEDED_PATTERN)) + } else if (completedContainer.getExitStatus != 0) { + logInfo("Container marked as failed: " + containerId + + ". Exit status: " + completedContainer.getExitStatus + + ". Diagnostics: " + completedContainer.getDiagnostics) numExecutorsFailed.incrementAndGet() } } @@ -428,6 +439,19 @@ private[yarn] abstract class YarnAllocator( } } + private val MEM_REGEX = "[0-9.]+ [KMG]B" + private val PMEM_EXCEEDED_PATTERN = + Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used") + private val VMEM_EXCEEDED_PATTERN = + Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used") + + def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = { + val matcher = pattern.matcher(diagnostics) + val diag = if (matcher.find()) " " + matcher.group() + "." else "" + ("Container killed by YARN for exceeding memory limits." + diag + + " Consider boosting spark.yarn.executor.memoryOverhead.") + } + protected def allocatedContainersOnHost(host: String): Int = { var retval = 0 allocatedHostToContainersMap.synchronized { diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala new file mode 100644 index 0000000000000..9fff63fb25156 --- /dev/null +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import org.apache.spark.deploy.yarn.MemLimitLogger._ +import org.scalatest.FunSuite + +class YarnAllocatorSuite extends FunSuite { + test("memory exceeded diagnostic regexes") { + val diagnostics = + "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " + + "beyond physical memory limits. Current usage: 2.1 MB of 2 GB physical memory used; " + + "5.8 GB of 4.2 GB virtual memory used. Killing container." + val vmemMsg = memLimitExceededLogMessage(diagnostics, VMEM_EXCEEDED_PATTERN) + val pmemMsg = memLimitExceededLogMessage(diagnostics, PMEM_EXCEEDED_PATTERN) + assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used.")) + assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used.")) + } +} \ No newline at end of file From adb6415c1d65d466a10c50e8dc6cb3bf2805ebdf Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 31 Oct 2014 10:28:19 -0700 Subject: [PATCH 106/115] [SPARK-4016] Allow user to show/hide UI metrics. This commit adds a set of checkboxes to the stage detail page that the user can use to show additional task metrics, including the GC time, result serialization time, result fetch time, and scheduler delay. All of these metrics are now hidden by default. This allows advanced users to look at more detailed metrics, without distracting the average user. This change also cleans up the stage detail page so that metrics are shown in the same order in the summary table as in the task table, and updates the metrics in both tables such that they contain the same set of metrics. The ability to remember a user's preferences for which metrics should be shown has been filed as SPARK-4024. Here's what the stage detail page looks like by default: ![image](https://cloud.githubusercontent.com/assets/1108612/4744322/3ebe319e-5a2f-11e4-891f-c792be79caa2.png) and once a user clicks "Show additional metrics" (note that all the metrics get checked by default): ![image](https://cloud.githubusercontent.com/assets/1108612/4744332/51e5abda-5a2f-11e4-8994-d0d3705ee05d.png) cc shivaram andrewor14 Author: Kay Ousterhout Closes #2867 from kayousterhout/SPARK-checkboxes and squashes the following commits: 6015913 [Kay Ousterhout] Added comment 08dee73 [Kay Ousterhout] Josh's usability comments 0940d61 [Kay Ousterhout] Style updates based on Andrew's review ef05ccd [Kay Ousterhout] Added tooltips d7cfaaf [Kay Ousterhout] Made list of add'l metrics collapsible. 70c1fb5 [Kay Ousterhout] [SPARK-4016] Allow user to show/hide UI metrics. --- .../spark/ui/static/additional-metrics.js | 53 ++++ .../org/apache/spark/ui/static/table.js | 35 +++ .../org/apache/spark/ui/static/webui.css | 30 +++ .../scala/org/apache/spark/ui/ToolTips.scala | 12 + .../scala/org/apache/spark/ui/UIUtils.scala | 44 ++-- .../org/apache/spark/ui/jobs/StagePage.scala | 242 ++++++++++++------ .../spark/ui/jobs/TaskDetailsClassNames.scala | 29 +++ 7 files changed, 350 insertions(+), 95 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js create mode 100644 core/src/main/resources/org/apache/spark/ui/static/table.js create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js new file mode 100644 index 0000000000000..c5936b5038ac9 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -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. + */ + +/* Register functions to show/hide columns based on checkboxes. These need + * to be registered after the page loads. */ +$(function() { + $("span.expand-additional-metrics").click(function(){ + // Expand the list of additional metrics. + var additionalMetricsDiv = $(this).parent().find('.additional-metrics'); + $(additionalMetricsDiv).toggleClass('collapsed'); + + // Switch the class of the arrow from open to closed. + $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-open'); + $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed'); + + // If clicking caused the metrics to expand, automatically check all options for additional + // metrics (don't trigger a click when collapsing metrics, because it leads to weird + // toggling behavior). + if (!$(additionalMetricsDiv).hasClass('collapsed')) { + $(this).parent().find('input:checkbox:not(:checked)').trigger('click'); + } + }); + + $("input:checkbox:not(:checked)").each(function() { + var column = "table ." + $(this).attr("name"); + $(column).hide(); + }); + + $("input:checkbox").click(function() { + var column = "table ." + $(this).attr("name"); + $(column).toggle(); + stripeTables(); + }); + + // Trigger a click on the checkbox if a user clicks the label next to it. + $("span.additional-metric-title").click(function() { + $(this).parent().find('input:checkbox').trigger('click'); + }); +}); diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js new file mode 100644 index 0000000000000..32187ba6e8df0 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* Adds background colors to stripe table rows. This is necessary (instead of using css or the + * table striping provided by bootstrap) to appropriately stripe tables with hidden rows. */ +function stripeTables() { + $("table.table-striped-custom").each(function() { + $(this).find("tr:not(:hidden)").each(function (index) { + if (index % 2 == 1) { + $(this).css("background-color", "#f9f9f9"); + } else { + $(this).css("background-color", "#ffffff"); + } + }); + }); +} + +/* Stripe all tables after pages finish loading. */ +$(function() { + stripeTables(); +}); 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 152bde5f6994f..a2220e761ac98 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 @@ -120,7 +120,37 @@ pre { border: none; } +span.expand-additional-metrics { + cursor: pointer; +} + +span.additional-metric-title { + cursor: pointer; +} + +.additional-metrics.collapsed { + display: none; +} + .tooltip { font-weight: normal; } +.arrow-open { + width: 0; + height: 0; + border-left: 5px solid transparent; + border-right: 5px solid transparent; + border-top: 5px solid black; + float: left; + margin-top: 6px; +} + +.arrow-closed { + width: 0; + height: 0; + border-top: 5px solid transparent; + border-bottom: 5px solid transparent; + border-left: 5px solid black; + display: inline-block; +} diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 9ced9b8107ebf..f02904df31fcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -31,4 +31,16 @@ private[spark] object ToolTips { val SHUFFLE_READ = """Bytes read from remote executors. Typically less than shuffle write bytes because this does not include shuffle data read locally.""" + + val GETTING_RESULT_TIME = + """Time that the driver spends fetching task results from workers. If this is large, consider + decreasing the amount of data returned from each task.""" + + val RESULT_SERIALIZATION_TIME = + """Time spent serializing the task result on the executor before sending it back to the + driver.""" + + val GC_TIME = + """Time that the executor spent paused for Java garbage collection while the task was + running.""" } 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 76714b1e6964f..3312671b6f885 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -20,13 +20,13 @@ package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.{Locale, Date} -import scala.xml.{Text, Node} +import scala.xml.{Node, Text} import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils extends Logging { - val TABLE_CLASS = "table table-bordered table-striped table-condensed sortable" + val TABLE_CLASS = "table table-bordered table-striped-custom table-condensed sortable" // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -160,6 +160,8 @@ private[spark] object UIUtils extends Logging { + + } /** Returns a spark page with correctly formatted headers */ @@ -240,7 +242,8 @@ private[spark] object UIUtils extends Logging { generateDataRow: T => Seq[Node], data: Iterable[T], fixedWidth: Boolean = false, - id: Option[String] = None): Seq[Node] = { + id: Option[String] = None, + headerClasses: Seq[String] = Seq.empty): Seq[Node] = { var listingTableClass = TABLE_CLASS if (fixedWidth) { @@ -248,20 +251,29 @@ private[spark] object UIUtils extends Logging { } val colWidth = 100.toDouble / headers.size val colWidthAttr = if (fixedWidth) colWidth + "%" else "" - val headerRow: Seq[Node] = { - // if none of the headers have "\n" in them - if (headers.forall(!_.contains("\n"))) { - // represent header as simple text - headers.map(h => {h}) + + def getClass(index: Int): String = { + if (index < headerClasses.size) { + headerClasses(index) } else { - // represent header text as list while respecting "\n" - headers.map { case h => - -
    - { h.split("\n").map { case t =>
  • {t}
  • } } -
- - } + "" + } + } + + val newlinesInHeader = headers.exists(_.contains("\n")) + def getHeaderContent(header: String): Seq[Node] = { + if (newlinesInHeader) { +
    + { header.split("\n").map { case t =>
  • {t}
  • } } +
+ } else { + Text(header) + } + } + + val headerRow: Seq[Node] = { + headers.view.zipWithIndex.map { x => + {getHeaderContent(x._1)} } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 2414e4c65237e..961224a300a70 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,10 +22,11 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, Unparsed} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.util.{Utils, Distribution} -import org.apache.spark.scheduler.AccumulableInfo +import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { @@ -57,7 +58,6 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val hasShuffleWrite = stageData.shuffleWriteBytes > 0 val hasBytesSpilled = stageData.memoryBytesSpilled > 0 && stageData.diskBytesSpilled > 0 - // scalastyle:off val summary =
    @@ -65,55 +65,102 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { Total task time across all tasks: {UIUtils.formatDuration(stageData.executorRunTime)} - {if (hasInput) + {if (hasInput) {
  • Input: {Utils.bytesToString(stageData.inputBytes)}
  • - } - {if (hasShuffleRead) + }} + {if (hasShuffleRead) {
  • Shuffle read: {Utils.bytesToString(stageData.shuffleReadBytes)}
  • - } - {if (hasShuffleWrite) + }} + {if (hasShuffleWrite) {
  • Shuffle write: {Utils.bytesToString(stageData.shuffleWriteBytes)}
  • - } - {if (hasBytesSpilled) -
  • - Shuffle spill (memory): - {Utils.bytesToString(stageData.memoryBytesSpilled)} -
  • -
  • - Shuffle spill (disk): - {Utils.bytesToString(stageData.diskBytesSpilled)} -
  • - } + }} + {if (hasBytesSpilled) { +
  • + Shuffle spill (memory): + {Utils.bytesToString(stageData.memoryBytesSpilled)} +
  • +
  • + Shuffle spill (disk): + {Utils.bytesToString(stageData.diskBytesSpilled)} +
  • + }}
- // scalastyle:on + + val showAdditionalMetrics = +
+ + + Show additional metrics + + +
+ val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") def accumulableRow(acc: AccumulableInfo) = val accumulableTable = UIUtils.listingTable(accumulableHeaders, accumulableRow, accumulables.values.toSeq) - val taskHeaders: Seq[String] = + val taskHeadersAndCssClasses: Seq[(String, String)] = Seq( - "Index", "ID", "Attempt", "Status", "Locality Level", "Executor ID / Host", - "Launch Time", "Duration", "GC Time", "Accumulators") ++ - {if (hasInput) Seq("Input") else Nil} ++ - {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ - {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++ - {if (hasBytesSpilled) Seq("Shuffle Spill (Memory)", "Shuffle Spill (Disk)") else Nil} ++ - Seq("Errors") + ("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""), + ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Accumulators", ""), + ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), + ("GC Time", TaskDetailsClassNames.GC_TIME), + ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), + ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ + {if (hasInput) Seq(("Input", "")) else Nil} ++ + {if (hasShuffleRead) Seq(("Shuffle Read", "")) else Nil} ++ + {if (hasShuffleWrite) Seq(("Write Time", ""), ("Shuffle Write", "")) else Nil} ++ + {if (hasBytesSpilled) Seq(("Shuffle Spill (Memory)", ""), ("Shuffle Spill (Disk)", "")) + else Nil} ++ + Seq(("Errors", "")) + + val unzipped = taskHeadersAndCssClasses.unzip val taskTable = UIUtils.listingTable( - taskHeaders, taskRow(hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) - + unzipped._1, taskRow(hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks, + headerClasses = unzipped._2) // Excludes tasks which failed and have incomplete metrics val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) @@ -122,18 +169,37 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { None } else { - val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.resultSerializationTime.toDouble + def getFormattedTimeQuantiles(times: Seq[Double]): Seq[Node] = { + Distribution(times).get.getQuantiles().map { millis => + + } } - val serializationQuantiles = - +: Distribution(serializationTimes). - get.getQuantiles().map(ms => ) val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.executorRunTime.toDouble } - val serviceQuantiles = +: Distribution(serviceTimes).get.getQuantiles() - .map(ms => ) + val serviceQuantiles = +: getFormattedTimeQuantiles(serviceTimes) + + val gcTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.jvmGCTime.toDouble + } + val gcQuantiles = + +: getFormattedTimeQuantiles(gcTimes) + + val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.resultSerializationTime.toDouble + } + val serializationQuantiles = + +: getFormattedTimeQuantiles(serializationTimes) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { @@ -142,76 +208,75 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { 0.0 } } - val gettingResultQuantiles = +: - Distribution(gettingResultTimes).get.getQuantiles().map { millis => - - } + val gettingResultQuantiles = + +: + getFormattedTimeQuantiles(gettingResultTimes) // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) => - val totalExecutionTime = { - if (info.gettingResultTime > 0) { - (info.gettingResultTime - info.launchTime).toDouble - } else { - (info.finishTime - info.launchTime).toDouble - } - } - totalExecutionTime - metrics.get.executorRunTime + getSchedulerDelay(info, metrics.get).toDouble } val schedulerDelayTitle = + title={ToolTips.SCHEDULER_DELAY} data-placement="right">Scheduler Delay val schedulerDelayQuantiles = schedulerDelayTitle +: - Distribution(schedulerDelays).get.getQuantiles().map { millis => - - } + getFormattedTimeQuantiles(schedulerDelays) - def getQuantileCols(data: Seq[Double]) = + def getFormattedSizeQuantiles(data: Seq[Double]) = Distribution(data).get.getQuantiles().map(d => ) val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble } - val inputQuantiles = +: getQuantileCols(inputSizes) + val inputQuantiles = +: getFormattedSizeQuantiles(inputSizes) val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } val shuffleReadQuantiles = +: - getQuantileCols(shuffleReadSizes) + getFormattedSizeQuantiles(shuffleReadSizes) val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } - val shuffleWriteQuantiles = +: getQuantileCols(shuffleWriteSizes) + val shuffleWriteQuantiles = +: + getFormattedSizeQuantiles(shuffleWriteSizes) val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.memoryBytesSpilled.toDouble } val memoryBytesSpilledQuantiles = +: - getQuantileCols(memoryBytesSpilledSizes) + getFormattedSizeQuantiles(memoryBytesSpilledSizes) val diskBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.diskBytesSpilled.toDouble } val diskBytesSpilledQuantiles = +: - getQuantileCols(diskBytesSpilledSizes) + getFormattedSizeQuantiles(diskBytesSpilledSizes) val listings: Seq[Seq[Node]] = Seq( - serializationQuantiles, - serviceQuantiles, - gettingResultQuantiles, - schedulerDelayQuantiles, - if (hasInput) inputQuantiles else Nil, - if (hasShuffleRead) shuffleReadQuantiles else Nil, - if (hasShuffleWrite) shuffleWriteQuantiles else Nil, - if (hasBytesSpilled) memoryBytesSpilledQuantiles else Nil, - if (hasBytesSpilled) diskBytesSpilledQuantiles else Nil) + {serviceQuantiles}, + {schedulerDelayQuantiles}, + {gcQuantiles}, + + {serializationQuantiles} + , + {gettingResultQuantiles}, + if (hasInput) {inputQuantiles} else Nil, + if (hasShuffleRead) {shuffleReadQuantiles} else Nil, + if (hasShuffleWrite) {shuffleWriteQuantiles} else Nil, + if (hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil, + if (hasBytesSpilled) {diskBytesSpilledQuantiles} else Nil) val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") - def quantileRow(data: Seq[Node]): Seq[Node] = {data} - Some(UIUtils.listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) + Some(UIUtils.listingTable( + quantileHeaders, identity[Seq[Node]], listings, fixedWidth = true)) } val executorTable = new ExecutorTable(stageId, stageAttemptId, parent) @@ -221,6 +286,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val content = summary ++ + showAdditionalMetrics ++

Summary Metrics for {numCompleted} Completed Tasks

++
{summaryTable.getOrElse("No tasks have reported metrics yet.")}
++

Aggregated Metrics by Executor

++ executorTable.toNodeSeq ++ @@ -241,8 +307,10 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { else metrics.map(_.executorRunTime).getOrElse(1L) val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") + val schedulerDelay = getSchedulerDelay(info, metrics.get) val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) + val gettingResultTime = info.gettingResultTime val maybeInput = metrics.flatMap(_.inputMetrics) val inputSortable = maybeInput.map(_.bytesRead.toString).getOrElse("") @@ -287,20 +355,25 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { - + - {if (hasInput) { } } + + private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = { + val totalExecutionTime = { + if (info.gettingResultTime > 0) { + (info.gettingResultTime - info.launchTime) + } else { + (info.finishTime - info.launchTime) + } + } + totalExecutionTime - metrics.executorRunTime + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala new file mode 100644 index 0000000000000..23d672cabda07 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala @@ -0,0 +1,29 @@ +/* + * 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.jobs + +/** + * Names of the CSS classes corresponding to each type of task detail. Used to allow users + * to optionally show/hide columns. + */ +private object TaskDetailsClassNames { + val SCHEDULER_DELAY = "scheduler_delay" + val GC_TIME = "gc_time" + val RESULT_SERIALIZATION_TIME = "serialization_time" + val GETTING_RESULT_TIME = "getting_result_time" +} From 7c41d135709c148d4fa3a1b06b5905715c970519 Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 31 Oct 2014 11:27:59 -0700 Subject: [PATCH 107/115] [SPARK-3826][SQL]enable hive-thriftserver to support hive-0.13.1 In #2241 hive-thriftserver is not enabled. This patch enable hive-thriftserver to support hive-0.13.1 by using a shim layer refer to #2241. 1 A light shim layer(code in sql/hive-thriftserver/hive-version) for each different hive version to handle api compatibility 2 New pom profiles "hive-default" and "hive-versions"(copy from #2241) to activate different hive version 3 SBT cmd for different version as follows: hive-0.12.0 --- sbt/sbt -Phive,hadoop-2.3 -Phive-0.12.0 assembly hive-0.13.1 --- sbt/sbt -Phive,hadoop-2.3 -Phive-0.13.1 assembly 4 Since hive-thriftserver depend on hive subproject, this patch should be merged with #2241 to enable hive-0.13.1 for hive-thriftserver Author: wangfei Author: scwf Closes #2685 from scwf/shim-thriftserver1 and squashes the following commits: f26f3be [wangfei] remove clean to save time f5cac74 [wangfei] remove local hivecontext test 578234d [wangfei] use new shaded hive 18fb1ff [wangfei] exclude kryo in hive pom fa21d09 [wangfei] clean package assembly/assembly 8a4daf2 [wangfei] minor fix 0d7f6cf [wangfei] address comments f7c93ae [wangfei] adding build with hive 0.13 before running tests bcf943f [wangfei] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver1 c359822 [wangfei] reuse getCommandProcessor in hiveshim 52674a4 [scwf] sql/hive included since examples depend on it 3529e98 [scwf] move hive module to hive profile f51ff4e [wangfei] update and fix conflicts f48d3a5 [scwf] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver1 41f727b [scwf] revert pom changes 13afde0 [scwf] fix small bug 4b681f4 [scwf] enable thriftserver in profile hive-0.13.1 0bc53aa [scwf] fixed when result filed is null dfd1c63 [scwf] update run-tests to run hive-0.12.0 default now c6da3ce [scwf] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver 7c66b8e [scwf] update pom according spark-2706 ae47489 [scwf] update and fix conflicts --- assembly/pom.xml | 6 - dev/run-tests | 13 +- pom.xml | 29 +- python/pyspark/sql.py | 27 -- sql/hive-thriftserver/pom.xml | 18 ++ ...ver.scala => AbstractSparkSQLDriver.scala} | 18 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 6 +- .../thriftserver/SparkSQLCLIService.scala | 19 +- .../server/SparkSQLOperationManager.scala | 169 +---------- .../spark/sql/hive/thriftserver/Shim12.scala | 225 +++++++++++++++ .../spark/sql/hive/thriftserver/Shim13.scala | 267 ++++++++++++++++++ sql/hive/pom.xml | 4 + 12 files changed, 571 insertions(+), 230 deletions(-) rename sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/{SparkSQLDriver.scala => AbstractSparkSQLDriver.scala} (86%) create mode 100644 sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala create mode 100644 sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala diff --git a/assembly/pom.xml b/assembly/pom.xml index 11d4bea9361ab..9e8525dd46157 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -201,12 +201,6 @@ spark-hive_${scala.binary.version} ${project.version} - - - - - hive-0.12.0 - org.apache.spark spark-hive-thriftserver_${scala.binary.version} diff --git a/dev/run-tests b/dev/run-tests index 972c8c8a21567..0e9eefa76a18b 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -142,17 +142,24 @@ CURRENT_BLOCK=$BLOCK_BUILD # We always build with Hive because the PySpark Spark SQL tests need it. BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" - echo "[info] Building Spark with these arguments: $BUILD_MVN_PROFILE_ARGS" # NOTE: echo "q" is needed because sbt on encountering a build file with failure #+ (either resolution or compilation) prompts the user for input either q, r, etc #+ to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a + # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a #+ single argument! # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + # First build with 0.12 to ensure patches do not break the hive 12 build + echo "[info] Compile with hive 0.12" echo -e "q\n" \ - | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly \ + | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean hive/compile hive-thriftserver/compile \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + + # Then build with default version(0.13.1) because tests are based on this version + echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS -Phive" + echo -e "q\n" \ + | sbt/sbt $SBT_MAVEN_PROFILES_ARGS -Phive package assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } diff --git a/pom.xml b/pom.xml index 379274d0b16e3..42fdbb9e097db 100644 --- a/pom.xml +++ b/pom.xml @@ -129,7 +129,7 @@ 1.4.0 3.4.5 - 0.13.1 + 0.13.1a 0.13.1 10.10.1.1 @@ -240,6 +240,18 @@ false + + + spark-staging-hive13 + Spring Staging Repository Hive 13 + https://oss.sonatype.org/content/repositories/orgspark-project-1089/ + + true + + + false + + @@ -908,9 +920,9 @@ by Spark SQL for code generation. --> - org.scalamacros - paradise_${scala.version} - ${scala.macros.version} + org.scalamacros + paradise_${scala.version} + ${scala.macros.version} @@ -1314,14 +1326,19 @@ - hive-0.12.0 + hive false - sql/hive-thriftserver + + + hive-0.12.0 + + false + 0.12.0-protobuf-2.5 0.12.0 diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 93fd9d49096b8..f0bd3cbd985da 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1400,33 +1400,6 @@ def hql(self, hqlQuery): class LocalHiveContext(HiveContext): - """Starts up an instance of hive where metadata is stored locally. - - An in-process metadata data is created with data stored in ./metadata. - Warehouse data is stored in in ./warehouse. - - >>> import os - >>> hiveCtx = LocalHiveContext(sc) - >>> try: - ... supress = hiveCtx.sql("DROP TABLE src") - ... except Exception: - ... pass - >>> kv1 = os.path.join(os.environ["SPARK_HOME"], - ... 'examples/src/main/resources/kv1.txt') - >>> supress = hiveCtx.sql( - ... "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - >>> supress = hiveCtx.sql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src" - ... % kv1) - >>> results = hiveCtx.sql("FROM src SELECT value" - ... ).map(lambda r: int(r.value.split('_')[1])) - >>> num = results.count() - >>> reduce_sum = results.reduce(lambda x, y: x + y) - >>> num - 500 - >>> reduce_sum - 130091 - """ - def __init__(self, sparkContext, sqlContext=None): HiveContext.__init__(self, sparkContext, sqlContext) warnings.warn("LocalHiveContext is deprecated. " diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 124fc107cb8aa..8db3010624100 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -70,6 +70,24 @@ org.scalatest scalatest-maven-plugin + + org.codehaus.mojo + build-helper-maven-plugin + + + add-default-sources + generate-sources + + add-source + + + + v${hive.version.short}/src/main/scala + + + + + org.apache.maven.plugins maven-deploy-plugin 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/AbstractSparkSQLDriver.scala similarity index 86% rename from sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala rename to sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala index a5c457c677564..fcb302edbffa8 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/AbstractSparkSQLDriver.scala @@ -29,11 +29,11 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.Logging import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) - extends Driver with Logging { +private[hive] abstract class AbstractSparkSQLDriver( + val context: HiveContext = SparkSQLEnv.hiveContext) extends Driver with Logging { - private var tableSchema: Schema = _ - private var hiveResponse: Seq[String] = _ + private[hive] var tableSchema: Schema = _ + private[hive] var hiveResponse: Seq[String] = _ override def init(): Unit = { } @@ -74,16 +74,6 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo override def getSchema: Schema = tableSchema - override def getResults(res: JArrayList[String]): Boolean = { - if (hiveResponse == null) { - false - } else { - res.addAll(hiveResponse) - hiveResponse = null - true - } - } - override def destroy() { super.destroy() hiveResponse = null diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 7ba4564602ecd..2cd02ae9269f5 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -38,6 +38,8 @@ import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket import org.apache.spark.Logging +import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.hive.thriftserver.HiveThriftServerShim private[hive] object SparkSQLCLIDriver { private var prompt = "spark-sql" @@ -116,7 +118,7 @@ private[hive] object SparkSQLCLIDriver { } } - if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { + if (!sessionState.isRemoteMode) { // Hadoop-20 and above - we need to augment classpath using hiveconf // components. // See also: code in ExecDriver.java @@ -258,7 +260,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } else { var ret = 0 val hconf = conf.asInstanceOf[HiveConf] - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) + val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hconf) if (proc != null) { if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 42cbf363b274f..a78311fc48635 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -24,6 +24,7 @@ import java.util.{List => JList} import javax.security.auth.login.LoginException import org.apache.commons.logging.Log +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hive.service.Service.STATE @@ -44,15 +45,17 @@ private[hive] class SparkSQLCLIService(hiveContext: HiveContext) val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) setSuperField(this, "sessionManager", sparkSqlSessionManager) addService(sparkSqlSessionManager) + var sparkServiceUGI: UserGroupInformation = null - try { - HiveAuthFactory.loginFromKeytab(hiveConf) - val serverUserName = ShimLoader.getHadoopShims - .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) - setSuperField(this, "serverUserName", serverUserName) - } catch { - case e @ (_: IOException | _: LoginException) => - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) + if (ShimLoader.getHadoopShims().isSecurityEnabled()) { + try { + HiveAuthFactory.loginFromKeytab(hiveConf) + sparkServiceUGI = ShimLoader.getHadoopShims.getUGIForConf(hiveConf) + HiveThriftServerShim.setServerUserName(sparkServiceUGI, this) + } catch { + case e @ (_: IOException | _: LoginException) => + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) + } } initCompositeService(hiveConf) 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 accf61576b804..2a4f24132cc5e 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 @@ -17,24 +17,15 @@ package org.apache.spark.sql.hive.thriftserver.server -import java.sql.Timestamp import java.util.{Map => JMap} +import scala.collection.mutable.Map -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, Map} -import scala.math.{random, round} - -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging -import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} -import org.apache.spark.sql.catalyst.plans.logical.SetCommand -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.{SparkExecuteStatementOperation, ReflectionUtils} /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -54,158 +45,8 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) confOverlay: JMap[String, String], async: Boolean): ExecuteStatementOperation = synchronized { - val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { - private var result: SchemaRDD = _ - private var iter: Iterator[SparkRow] = _ - private var dataTypes: Array[DataType] = _ - - def close(): Unit = { - // RDDs will be cleaned automatically upon garbage collection. - logDebug("CLOSING") - } - - def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { - if (!iter.hasNext) { - new RowSet() - } else { - // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int - val maxRows = maxRowsL.toInt - var curRow = 0 - var rowSet = new ArrayBuffer[Row](maxRows.min(1024)) - - while (curRow < maxRows && iter.hasNext) { - val sparkRow = iter.next() - val row = new Row() - var curCol = 0 - - while (curCol < sparkRow.length) { - if (sparkRow.isNullAt(curCol)) { - addNullColumnValue(sparkRow, row, curCol) - } else { - addNonNullColumnValue(sparkRow, row, curCol) - } - curCol += 1 - } - rowSet += row - curRow += 1 - } - new RowSet(rowSet, 0) - } - } - - def addNonNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { - dataTypes(ordinal) match { - case StringType => - to.addString(from(ordinal).asInstanceOf[String]) - case IntegerType => - to.addColumnValue(ColumnValue.intValue(from.getInt(ordinal))) - case BooleanType => - to.addColumnValue(ColumnValue.booleanValue(from.getBoolean(ordinal))) - case DoubleType => - to.addColumnValue(ColumnValue.doubleValue(from.getDouble(ordinal))) - case FloatType => - to.addColumnValue(ColumnValue.floatValue(from.getFloat(ordinal))) - case DecimalType => - val hiveDecimal = from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal - to.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) - case LongType => - to.addColumnValue(ColumnValue.longValue(from.getLong(ordinal))) - case ByteType => - to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) - case ShortType => - to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) - case TimestampType => - to.addColumnValue( - ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = result - .queryExecution - .asInstanceOf[HiveContext#QueryExecution] - .toHiveString((from.get(ordinal), dataTypes(ordinal))) - to.addColumnValue(ColumnValue.stringValue(hiveString)) - } - } - - def addNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { - dataTypes(ordinal) match { - case StringType => - to.addString(null) - case IntegerType => - to.addColumnValue(ColumnValue.intValue(null)) - case BooleanType => - to.addColumnValue(ColumnValue.booleanValue(null)) - case DoubleType => - to.addColumnValue(ColumnValue.doubleValue(null)) - case FloatType => - to.addColumnValue(ColumnValue.floatValue(null)) - case DecimalType => - to.addColumnValue(ColumnValue.stringValue(null: HiveDecimal)) - case LongType => - to.addColumnValue(ColumnValue.longValue(null)) - case ByteType => - to.addColumnValue(ColumnValue.byteValue(null)) - case ShortType => - to.addColumnValue(ColumnValue.shortValue(null)) - case TimestampType => - to.addColumnValue(ColumnValue.timestampValue(null)) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - to.addColumnValue(ColumnValue.stringValue(null: String)) - } - } - - def getResultSetSchema: TableSchema = { - logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") - if (result.queryExecution.analyzed.output.size == 0) { - new TableSchema(new FieldSchema("Result", "string", "") :: Nil) - } else { - val schema = result.queryExecution.analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - new TableSchema(schema) - } - } - - def run(): Unit = { - logInfo(s"Running query '$statement'") - setState(OperationState.RUNNING) - try { - result = hiveContext.sql(statement) - logDebug(result.queryExecution.toString()) - result.queryExecution.logical match { - case SetCommand(Some((SQLConf.THRIFTSERVER_POOL, Some(value)))) => - sessionToActivePool(parentSession) = value - logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") - case _ => - } - - val groupId = round(random * 1000000).toString - hiveContext.sparkContext.setJobGroup(groupId, statement) - sessionToActivePool.get(parentSession).foreach { pool => - hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) - } - iter = { - val resultRdd = result.queryExecution.toRdd - val useIncrementalCollect = - hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean - if (useIncrementalCollect) { - resultRdd.toLocalIterator - } else { - resultRdd.collect().iterator - } - } - dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray - setHasResultSet(true) - } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - logError("Error executing query:",e) - throw new HiveSQLException(e.toString) - } - setState(OperationState.FINISHED) - } - } - + val operation = new SparkExecuteStatementOperation(parentSession, statement, confOverlay)( + hiveContext, sessionToActivePool) handleToOperation.put(operation.getHandle, operation) operation } diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala new file mode 100644 index 0000000000000..bbd727c686bbc --- /dev/null +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -0,0 +1,225 @@ +/* + * 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.thriftserver + +import java.sql.Timestamp +import java.util.{ArrayList => JArrayList, Map => JMap} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, Map => SMap} +import scala.math._ + +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.ExecuteStatementOperation +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.plans.logical.SetCommand +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} +import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveContext} +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +/** + * A compatibility layer for interacting with Hive version 0.12.0. + */ +private[thriftserver] object HiveThriftServerShim { + val version = "0.12.0" + + def setServerUserName(sparkServiceUGI: UserGroupInformation, sparkCliService:SparkSQLCLIService) = { + val serverUserName = ShimLoader.getHadoopShims.getShortUserName(sparkServiceUGI) + setSuperField(sparkCliService, "serverUserName", serverUserName) + } +} + +private[hive] class SparkSQLDriver(val _context: HiveContext = SparkSQLEnv.hiveContext) + extends AbstractSparkSQLDriver(_context) { + override def getResults(res: JArrayList[String]): Boolean = { + if (hiveResponse == null) { + false + } else { + res.addAll(hiveResponse) + hiveResponse = null + true + } + } +} + +private[hive] class SparkExecuteStatementOperation( + parentSession: HiveSession, + statement: String, + confOverlay: JMap[String, String])( + hiveContext: HiveContext, + sessionToActivePool: SMap[HiveSession, String]) extends ExecuteStatementOperation( + parentSession, statement, confOverlay) with Logging { + private var result: SchemaRDD = _ + private var iter: Iterator[SparkRow] = _ + private var dataTypes: Array[DataType] = _ + + def close(): Unit = { + // RDDs will be cleaned automatically upon garbage collection. + logDebug("CLOSING") + } + + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + if (!iter.hasNext) { + new RowSet() + } else { + // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int + val maxRows = maxRowsL.toInt + var curRow = 0 + var rowSet = new ArrayBuffer[Row](maxRows.min(1024)) + + while (curRow < maxRows && iter.hasNext) { + val sparkRow = iter.next() + val row = new Row() + var curCol = 0 + + while (curCol < sparkRow.length) { + if (sparkRow.isNullAt(curCol)) { + addNullColumnValue(sparkRow, row, curCol) + } else { + addNonNullColumnValue(sparkRow, row, curCol) + } + curCol += 1 + } + rowSet += row + curRow += 1 + } + new RowSet(rowSet, 0) + } + } + + def addNonNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to.addString(from(ordinal).asInstanceOf[String]) + case IntegerType => + to.addColumnValue(ColumnValue.intValue(from.getInt(ordinal))) + case BooleanType => + to.addColumnValue(ColumnValue.booleanValue(from.getBoolean(ordinal))) + case DoubleType => + to.addColumnValue(ColumnValue.doubleValue(from.getDouble(ordinal))) + case FloatType => + to.addColumnValue(ColumnValue.floatValue(from.getFloat(ordinal))) + case DecimalType => + val hiveDecimal = from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal + to.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) + case LongType => + to.addColumnValue(ColumnValue.longValue(from.getLong(ordinal))) + case ByteType => + to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) + case ShortType => + to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) + case TimestampType => + to.addColumnValue( + ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + val hiveString = result + .queryExecution + .asInstanceOf[HiveContext#QueryExecution] + .toHiveString((from.get(ordinal), dataTypes(ordinal))) + to.addColumnValue(ColumnValue.stringValue(hiveString)) + } + } + + def addNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to.addString(null) + case IntegerType => + to.addColumnValue(ColumnValue.intValue(null)) + case BooleanType => + to.addColumnValue(ColumnValue.booleanValue(null)) + case DoubleType => + to.addColumnValue(ColumnValue.doubleValue(null)) + case FloatType => + to.addColumnValue(ColumnValue.floatValue(null)) + case DecimalType => + to.addColumnValue(ColumnValue.stringValue(null: HiveDecimal)) + case LongType => + to.addColumnValue(ColumnValue.longValue(null)) + case ByteType => + to.addColumnValue(ColumnValue.byteValue(null)) + case ShortType => + to.addColumnValue(ColumnValue.shortValue(null)) + case TimestampType => + to.addColumnValue(ColumnValue.timestampValue(null)) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + to.addColumnValue(ColumnValue.stringValue(null: String)) + } + } + + def getResultSetSchema: TableSchema = { + logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + def run(): Unit = { + logInfo(s"Running query '$statement'") + setState(OperationState.RUNNING) + try { + result = hiveContext.sql(statement) + logDebug(result.queryExecution.toString()) + result.queryExecution.logical match { + case SetCommand(Some((SQLConf.THRIFTSERVER_POOL, Some(value)))) => + sessionToActivePool(parentSession) = value + logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") + case _ => + } + + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + sessionToActivePool.get(parentSession).foreach { pool => + hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) + } + iter = { + val resultRdd = result.queryExecution.toRdd + val useIncrementalCollect = + hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean + if (useIncrementalCollect) { + resultRdd.toLocalIterator + } else { + resultRdd.collect().iterator + } + } + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + setHasResultSet(true) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logError("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + setState(OperationState.FINISHED) + } +} diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala new file mode 100644 index 0000000000000..e59681bfbe43e --- /dev/null +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -0,0 +1,267 @@ +/* + * 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.thriftserver + +import java.security.PrivilegedExceptionAction +import java.sql.Timestamp +import java.util.concurrent.Future +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, Map => SMap} +import scala.math._ + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.ExecuteStatementOperation +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.{Row => SparkRow, SchemaRDD} +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +/** + * A compatibility layer for interacting with Hive version 0.12.0. + */ +private[thriftserver] object HiveThriftServerShim { + val version = "0.13.1" + + def setServerUserName(sparkServiceUGI: UserGroupInformation, sparkCliService:SparkSQLCLIService) = { + setSuperField(sparkCliService, "serviceUGI", sparkServiceUGI) + } +} + +private[hive] class SparkSQLDriver(val _context: HiveContext = SparkSQLEnv.hiveContext) + extends AbstractSparkSQLDriver(_context) { + override def getResults(res: JList[_]): Boolean = { + if (hiveResponse == null) { + false + } else { + res.asInstanceOf[JArrayList[String]].addAll(hiveResponse) + hiveResponse = null + true + } + } +} + +private[hive] class SparkExecuteStatementOperation( + parentSession: HiveSession, + statement: String, + confOverlay: JMap[String, String], + runInBackground: Boolean = true)( + hiveContext: HiveContext, + sessionToActivePool: SMap[HiveSession, String]) extends ExecuteStatementOperation( + parentSession, statement, confOverlay, runInBackground) with Logging { + + private var result: SchemaRDD = _ + private var iter: Iterator[SparkRow] = _ + private var dataTypes: Array[DataType] = _ + + private def runInternal(cmd: String) = { + try { + result = hiveContext.sql(cmd) + logDebug(result.queryExecution.toString()) + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + iter = { + val resultRdd = result.queryExecution.toRdd + val useIncrementalCollect = + hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean + if (useIncrementalCollect) { + resultRdd.toLocalIterator + } else { + resultRdd.collect().iterator + } + } + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logError("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + } + + def close(): Unit = { + // RDDs will be cleaned automatically upon garbage collection. + logDebug("CLOSING") + } + + def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to += from.get(ordinal).asInstanceOf[String] + case IntegerType => + to += from.getInt(ordinal) + case BooleanType => + to += from.getBoolean(ordinal) + case DoubleType => + to += from.getDouble(ordinal) + case FloatType => + to += from.getFloat(ordinal) + case DecimalType => + to += from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal + case LongType => + to += from.getLong(ordinal) + case ByteType => + to += from.getByte(ordinal) + case ShortType => + to += from.getShort(ordinal) + case TimestampType => + to += from.get(ordinal).asInstanceOf[Timestamp] + case BinaryType => + to += from.get(ordinal).asInstanceOf[String] + case _: ArrayType => + to += from.get(ordinal).asInstanceOf[String] + case _: StructType => + to += from.get(ordinal).asInstanceOf[String] + case _: MapType => + to += from.get(ordinal).asInstanceOf[String] + } + } + + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + validateDefaultFetchOrientation(order) + assertState(OperationState.FINISHED) + setHasResultSet(true) + val reultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) + if (!iter.hasNext) { + reultRowSet + } else { + // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int + val maxRows = maxRowsL.toInt + var curRow = 0 + while (curRow < maxRows && iter.hasNext) { + val sparkRow = iter.next() + val row = ArrayBuffer[Any]() + var curCol = 0 + while (curCol < sparkRow.length) { + if (sparkRow.isNullAt(curCol)) { + row += null + } else { + addNonNullColumnValue(sparkRow, row, curCol) + } + curCol += 1 + } + reultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]]) + curRow += 1 + } + reultRowSet + } + } + + def getResultSetSchema: TableSchema = { + logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + private def getConfigForOperation: HiveConf = { + var sqlOperationConf: HiveConf = getParentSession.getHiveConf + if (!getConfOverlay.isEmpty || shouldRunAsync) { + sqlOperationConf = new HiveConf(sqlOperationConf) + import scala.collection.JavaConversions._ + for (confEntry <- getConfOverlay.entrySet) { + try { + sqlOperationConf.verifyAndSet(confEntry.getKey, confEntry.getValue) + } + catch { + case e: IllegalArgumentException => { + throw new HiveSQLException("Error applying statement specific settings", e) + } + } + } + } + return sqlOperationConf + } + + def run(): Unit = { + logInfo(s"Running query '$statement'") + val opConfig: HiveConf = getConfigForOperation + setState(OperationState.RUNNING) + setHasResultSet(true) + + if (!shouldRunAsync) { + runInternal(statement) + setState(OperationState.FINISHED) + } else { + val parentSessionState = SessionState.get + val sessionHive: Hive = Hive.get + val currentUGI: UserGroupInformation = ShimLoader.getHadoopShims.getUGIForConf(opConfig) + + val backgroundOperation: Runnable = new Runnable { + def run { + val doAsAction: PrivilegedExceptionAction[AnyRef] = + new PrivilegedExceptionAction[AnyRef] { + def run: AnyRef = { + Hive.set(sessionHive) + SessionState.setCurrentSessionState(parentSessionState) + try { + runInternal(statement) + } + catch { + case e: HiveSQLException => { + setOperationException(e) + logError("Error running hive query: ", e) + } + } + return null + } + } + try { + ShimLoader.getHadoopShims.doAs(currentUGI, doAsAction) + } + catch { + case e: Exception => { + setOperationException(new HiveSQLException(e)) + logError("Error running hive query as user : " + currentUGI.getShortUserName, e) + } + } + setState(OperationState.FINISHED) + } + } + + try { + val backgroundHandle: Future[_] = getParentSession.getSessionManager. + submitBackgroundOperation(backgroundOperation) + setBackgroundHandle(backgroundHandle) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logError("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + } + } +} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index db01363b4d629..67e36a951e506 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -65,6 +65,10 @@ commons-logging commons-logging + + com.esotericsoftware.kryo + kryo + From fa712b309c0e59943aae289dab629b34a13fe20e Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Fri, 31 Oct 2014 11:30:28 -0700 Subject: [PATCH 108/115] [SPARK-4077][SQL] Spark SQL return wrong values for valid string timestamp values In org.apache.hadoop.hive.serde2.io.TimestampWritable.set , if the next entry is null then current time stamp object is being reset. However because of this hiveinspectors:unwrap cannot use the same timestamp object without creating a copy. Author: Venkata Ramana G Author: Venkata Ramana Gollamudi Closes #3019 from gvramana/spark_4077 and squashes the following commits: 32d818f [Venkata Ramana Gollamudi] fixed check style fa01e71 [Venkata Ramana Gollamudi] cloned timestamp object as org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object --- .../spark/sql/hive/HiveInspectors.scala | 3 +++ .../resources/data/files/issue-4077-data.txt | 2 ++ .../hive/execution/HiveTableScanSuite.scala | 22 +++++++++++++++++++ 3 files changed, 27 insertions(+) create mode 100644 sql/hive/src/test/resources/data/files/issue-4077-data.txt diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index c6103a124df59..0439ab97d80be 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -91,6 +91,9 @@ private[hive] trait HiveInspectors { if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue case hdoi: HiveDecimalObjectInspector => if (data == null) null else BigDecimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) + // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object + // if next timestamp is null, so Timestamp object is cloned + case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone() case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) case li: ListObjectInspector => Option(li.getList(data)) diff --git a/sql/hive/src/test/resources/data/files/issue-4077-data.txt b/sql/hive/src/test/resources/data/files/issue-4077-data.txt new file mode 100644 index 0000000000000..18067b0a64c9c --- /dev/null +++ b/sql/hive/src/test/resources/data/files/issue-4077-data.txt @@ -0,0 +1,2 @@ +2014-12-11 00:00:00,1 +2014-12-11astring00:00:00,2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 2f3db95882093..54c0f017d4cb6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -18,6 +18,9 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.{Row, SchemaRDD} + +import org.apache.spark.util.Utils class HiveTableScanSuite extends HiveComparisonTest { @@ -47,4 +50,23 @@ class HiveTableScanSuite extends HiveComparisonTest { TestHive.sql("select KEY from tb where VALUE='just_for_test' limit 5").collect() TestHive.sql("drop table tb") } + + test("Spark-4077: timestamp query for null value") { + TestHive.sql("DROP TABLE IF EXISTS timestamp_query_null") + TestHive.sql( + """ + CREATE EXTERNAL TABLE timestamp_query_null (time TIMESTAMP,id INT) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + """.stripMargin) + val location = + Utils.getSparkClassLoader.getResource("data/files/issue-4077-data.txt").getFile() + + TestHive.sql(s"LOAD DATA LOCAL INPATH '$location' INTO TABLE timestamp_query_null") + assert(TestHive.sql("SELECT time from timestamp_query_null limit 2").collect() + === Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")),Row(null))) + TestHive.sql("DROP TABLE timestamp_query_null") + } + } From ea465af12ddae424af9b4e742c3d5aed2a0bc8ec Mon Sep 17 00:00:00 2001 From: ravipesala Date: Fri, 31 Oct 2014 11:33:20 -0700 Subject: [PATCH 109/115] [SPARK-4154][SQL] Query does not work if it has "not between " in Spark SQL and HQL if the query contains "not between" does not work like. SELECT * FROM src where key not between 10 and 20' Author: ravipesala Closes #3017 from ravipesala/SPARK-4154 and squashes the following commits: 65fc89e [ravipesala] Handled admin comments 32e6d42 [ravipesala] 'not between' is not working --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 6 ++++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 5 +++++ .../scala/org/apache/spark/sql/hive/HiveQl.scala | 13 +++++++++---- .../spark/sql/hive/execution/SQLQuerySuite.scala | 5 +++++ 4 files changed, 23 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 942b843104958..b1e7570f577c3 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -232,8 +232,10 @@ class SqlParser extends AbstractSparkSQLParser { | termExpression ~ (">=" ~> termExpression) ^^ { case e1 ~ e2 => GreaterThanOrEqual(e1, e2) } | termExpression ~ ("!=" ~> termExpression) ^^ { case e1 ~ e2 => Not(EqualTo(e1, e2)) } | termExpression ~ ("<>" ~> termExpression) ^^ { case e1 ~ e2 => Not(EqualTo(e1, e2)) } - | termExpression ~ (BETWEEN ~> termExpression) ~ (AND ~> termExpression) ^^ { - case e ~ el ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) + | termExpression ~ NOT.? ~ (BETWEEN ~> termExpression) ~ (AND ~> termExpression) ^^ { + case e ~ not ~ el ~ eu => + val betweenExpr: Expression = And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) + not.fold(betweenExpr)(f=> Not(betweenExpr)) } | termExpression ~ (RLIKE ~> termExpression) ^^ { case e1 ~ e2 => RLike(e1, e2) } | termExpression ~ (REGEXP ~> termExpression) ^^ { case e1 ~ e2 => RLike(e1, e2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4c36ca0f41174..4acd92d33d180 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -909,4 +909,9 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { """.stripMargin), (1 to 100).map(i => Seq(i, i, i))) } + + test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") { + checkAnswer(sql("SELECT key FROM testData WHERE key not between 0 and 10 order by key"), + (11 to 100).map(i => Seq(i))) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 9d9d68affa54b..a3573e6502de8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -985,15 +985,20 @@ private[hive] object HiveQl { In(nodeToExpr(value), list.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(BETWEEN(), Nil) :: - Token("KW_FALSE", Nil) :: + kw :: target :: minValue :: maxValue :: Nil) => val targetExpression = nodeToExpr(target) - And( - GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)), - LessThanOrEqual(targetExpression, nodeToExpr(maxValue))) + val betweenExpr = + And( + GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)), + LessThanOrEqual(targetExpression, nodeToExpr(maxValue))) + kw match { + case Token("KW_FALSE", Nil) => betweenExpr + case Token("KW_TRUE", Nil) => Not(betweenExpr) + } /* Boolean Logic */ case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) 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 4f96a327ee2c7..76a0ec01a6075 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 @@ -158,4 +158,9 @@ class SQLQuerySuite extends QueryTest { sql("SELECT case when ~1=-2 then 1 else 0 end FROM src"), sql("SELECT 1 FROM src").collect().toSeq) } + + test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") { + checkAnswer(sql("SELECT key FROM src WHERE key not between 0 and 10 order by key"), + sql("SELECT key FROM src WHERE key between 11 and 500 order by key").collect().toSeq) + } } From 23468e7e96bf047ba53806352558b9d661567b23 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 31 Oct 2014 11:34:51 -0700 Subject: [PATCH 110/115] [SPARK-2220][SQL] Fixes remaining Hive commands This PR adds support for the `ADD FILE` Hive command, and removes `ShellCommand` and `SourceCommand`. The reason is described in [this SPARK-2220 comment](https://issues.apache.org/jira/browse/SPARK-2220?focusedCommentId=14191841&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14191841). Author: Cheng Lian Closes #3038 from liancheng/hive-commands and squashes the following commits: 6db61e0 [Cheng Lian] Fixes remaining Hive commands --- .../spark/sql/catalyst/SparkSQLParser.scala | 14 +----------- .../sql/catalyst/plans/logical/commands.scala | 22 +++++++++---------- .../spark/sql/hive/HiveStrategies.scala | 2 ++ .../spark/sql/hive/execution/commands.scala | 16 ++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 19 +++++++++++++--- 5 files changed, 45 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala index 12e8346a6445d..f5c19ee69c37a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala @@ -137,7 +137,6 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr protected val LAZY = Keyword("LAZY") protected val SET = Keyword("SET") protected val TABLE = Keyword("TABLE") - protected val SOURCE = Keyword("SOURCE") protected val UNCACHE = Keyword("UNCACHE") protected implicit def asParser(k: Keyword): Parser[String] = @@ -152,8 +151,7 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr override val lexical = new SqlLexical(reservedWords) - override protected lazy val start: Parser[LogicalPlan] = - cache | uncache | set | shell | source | others + override protected lazy val start: Parser[LogicalPlan] = cache | uncache | set | others private lazy val cache: Parser[LogicalPlan] = CACHE ~> LAZY.? ~ (TABLE ~> ident) ~ (AS ~> restInput).? ^^ { @@ -171,16 +169,6 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr case input => SetCommandParser(input) } - private lazy val shell: Parser[LogicalPlan] = - "!" ~> restInput ^^ { - case input => ShellCommand(input.trim) - } - - private lazy val source: Parser[LogicalPlan] = - SOURCE ~> restInput ^^ { - case input => SourceCommand(input.trim) - } - private lazy val others: Parser[LogicalPlan] = wholeInput ^^ { case input => fallback(input) 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 b8ba2ee428a20..1d513d7789763 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.types.StringType /** @@ -41,6 +41,15 @@ case class NativeCommand(cmd: String) extends Command { /** * Commands of the form "SET [key [= value] ]". */ +case class DFSCommand(kv: Option[(String, Option[String])]) extends Command { + override def output = Seq( + AttributeReference("DFS output", StringType, nullable = false)()) +} + +/** + * + * Commands of the form "SET [key [= value] ]". + */ case class SetCommand(kv: Option[(String, Option[String])]) extends Command { override def output = Seq( AttributeReference("", StringType, nullable = false)()) @@ -81,14 +90,3 @@ case class DescribeCommand( AttributeReference("data_type", StringType, nullable = false)(), AttributeReference("comment", StringType, nullable = false)()) } - -/** - * Returned for the "! shellCommand" command - */ -case class ShellCommand(cmd: String) extends Command - - -/** - * Returned for the "SOURCE file" command - */ -case class SourceCommand(filePath: String) extends Command 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 e59d4d536a0af..3207ad81d9571 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 @@ -206,6 +206,8 @@ private[hive] trait HiveStrategies { case hive.AddJar(path) => execution.AddJar(path) :: Nil + case hive.AddFile(path) => execution.AddFile(path) :: Nil + case hive.AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil case describe: logical.DescribeCommand => 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 0fc674af31885..903075edf7e04 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 @@ -76,3 +76,19 @@ case class AddJar(path: String) extends LeafNode with Command { Seq.empty[Row] } } + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class AddFile(path: String) extends LeafNode with Command { + def hiveContext = sqlContext.asInstanceOf[HiveContext] + + override def output = Seq.empty + + override protected lazy val sideEffectResult: Seq[Row] = { + hiveContext.runSqlHive(s"ADD FILE $path") + hiveContext.sparkContext.addFile(path) + Seq.empty[Row] + } +} 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 ffe1f0b90fcd0..5918f888c8f4c 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 @@ -17,11 +17,13 @@ package org.apache.spark.sql.hive.execution +import java.io.File + import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.spark.SparkException +import org.apache.spark.{SparkFiles, SparkException} import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive @@ -569,7 +571,7 @@ class HiveQuerySuite extends HiveComparisonTest { |WITH serdeproperties('s1'='9') """.stripMargin) } - // Now only verify 0.12.0, and ignore other versions due to binary compatability + // Now only verify 0.12.0, and ignore other versions due to binary compatibility // current TestSerDe.jar is from 0.12.0 if (HiveShim.version == "0.12.0") { sql(s"ADD JAR $testJar") @@ -581,6 +583,17 @@ class HiveQuerySuite extends HiveComparisonTest { sql("DROP TABLE alter1") } + test("ADD FILE command") { + val testFile = TestHive.getHiveFile("data/files/v1.txt").getCanonicalFile + sql(s"ADD FILE $testFile") + + val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { _ => + Iterator.single(new File(SparkFiles.get("v1.txt")).canRead) + } + + assert(checkAddFileRDD.first()) + } + case class LogEntry(filename: String, message: String) case class LogFile(name: String) @@ -816,7 +829,7 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("select from thrift based table", "SELECT * from src_thrift") - + // Put tests that depend on specific Hive settings before these last two test, // since they modify /clear stuff. } From a68ecf32812b5ef150e4b455b0f13b3fe120dc80 Mon Sep 17 00:00:00 2001 From: Mark Mims Date: Fri, 31 Oct 2014 11:41:03 -0700 Subject: [PATCH 111/115] [SPARK-4141] Hide Accumulators column on stage page when no accumulators exist WebUI Author: Mark Mims This patch had conflicts when merged, resolved by Committer: Josh Rosen Closes #3031 from mmm/remove-accumulators-col and squashes the following commits: 6141cb3 [Mark Mims] reformat to satisfy scalastyle linelength. build failed from jenkins https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22604/ 390893b [Mark Mims] cleanup c28c449 [Mark Mims] looking much better now... minimal explicit formatting. Now, see if any sort keys make sense fb72156 [Mark Mims] mimic hasInput. The basics work here, but wanna clean this up with maybeAccumulators for column content --- .../org/apache/spark/ui/jobs/StagePage.scala | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 961224a300a70..bf45272aefde4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -53,6 +53,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val numCompleted = tasks.count(_.taskInfo.finished) val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables + val hasAccumulators = accumulables.size > 0 val hasInput = stageData.inputBytes > 0 val hasShuffleRead = stageData.shuffleReadBytes > 0 val hasShuffleWrite = stageData.shuffleWriteBytes > 0 @@ -144,11 +145,12 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val taskHeadersAndCssClasses: Seq[(String, String)] = Seq( ("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""), - ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Accumulators", ""), + ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), ("GC Time", TaskDetailsClassNames.GC_TIME), ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ + {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++ {if (hasInput) Seq(("Input", "")) else Nil} ++ {if (hasShuffleRead) Seq(("Shuffle Read", "")) else Nil} ++ {if (hasShuffleWrite) Seq(("Write Time", ""), ("Shuffle Write", "")) else Nil} ++ @@ -159,7 +161,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val unzipped = taskHeadersAndCssClasses.unzip val taskTable = UIUtils.listingTable( - unzipped._1, taskRow(hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks, + unzipped._1, + taskRow(hasAccumulators, hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), + tasks, headerClasses = unzipped._2) // Excludes tasks which failed and have incomplete metrics val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) @@ -298,6 +302,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { } def taskRow( + hasAccumulators: Boolean, hasInput: Boolean, hasShuffleRead: Boolean, hasShuffleWrite: Boolean, @@ -312,6 +317,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = info.gettingResultTime + val maybeAccumulators = info.accumulables + val accumulatorsReadable = maybeAccumulators.map{acc => s"${acc.name}: ${acc.update.get}"} + val maybeInput = metrics.flatMap(_.inputMetrics) val inputSortable = maybeInput.map(_.bytesRead.toString).getOrElse("") val inputReadable = maybeInput @@ -355,10 +363,6 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { - + {if (hasAccumulators) { + + }} {if (hasInput) {
{acc.name}{acc.value}
{UIUtils.formatDuration(millis.toLong)}Result serialization time{UIUtils.formatDuration(ms.toLong)}Duration{UIUtils.formatDuration(ms.toLong)}Duration + GC Time + + + + Result Serialization Time + + Time spent fetching task results{UIUtils.formatDuration(millis.toLong)} + + Getting Result Time + + Scheduler delay{UIUtils.formatDuration(millis.toLong)}{Utils.bytesToString(d.toLong)}InputInputShuffle Read (Remote)Shuffle WriteShuffle WriteShuffle spill (memory)Shuffle spill (disk)
{formatDuration} - {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} - {Unparsed( - info.accumulables.map{acc => s"${acc.name}: ${acc.update.get}"}.mkString("
") - )} + info.accumulables.map{acc => s"${acc.name}: ${acc.update.get}"}.mkString("
"))} +
+ {UIUtils.formatDuration(schedulerDelay.toLong)} {inputReadable} @@ -333,4 +406,15 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
{formatDuration} - {Unparsed( - info.accumulables.map{acc => s"${acc.name}: ${acc.update.get}"}.mkString("
"))} -
{UIUtils.formatDuration(schedulerDelay.toLong)} @@ -374,6 +378,11 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { class={TaskDetailsClassNames.GETTING_RESULT_TIME}> {UIUtils.formatDuration(gettingResultTime)} + {Unparsed(accumulatorsReadable.mkString("
"))} +
{inputReadable} From f1e7361f66fc24ae79bf48cc25f7fe395b016285 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 31 Oct 2014 12:07:48 -0700 Subject: [PATCH 112/115] [SPARK-4150][PySpark] return self in rdd.setName Then we can do `rdd.setName('abc').cache().count()`. Author: Xiangrui Meng Closes #3011 from mengxr/rdd-setname and squashes the following commits: 10d0d60 [Xiangrui Meng] update test 4ac3bbd [Xiangrui Meng] return self in rdd.setName --- python/pyspark/rdd.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 15be4bfec92f9..550c9dd80522f 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1867,11 +1867,11 @@ def setName(self, name): Assign a name to this RDD. >>> rdd1 = sc.parallelize([1,2]) - >>> rdd1.setName('RDD1') - >>> rdd1.name() + >>> rdd1.setName('RDD1').name() 'RDD1' """ self._jrdd.setName(name) + return self def toDebugString(self): """ From 55ab77707891408e635843cf80079747625bd28e Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 31 Oct 2014 12:39:52 -0700 Subject: [PATCH 113/115] [SPARK-3870] EOL character enforcement We have shell scripts and Windows batch files, so we should enforce proper EOL character. Author: Kousuke Saruta Closes #2726 from sarutak/eol-enforcement and squashes the following commits: 9748c3f [Kousuke Saruta] Fixed make.bat 252de89 [Kousuke Saruta] Removed extra characters from make.bat 5b81c00 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into eol-enforcement 8633ed2 [Kousuke Saruta] merge branch 'master' of git://git.apache.org/spark into eol-enforcement 5d630d8 [Kousuke Saruta] Merged ba10797 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into eol-enforcement 7407515 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into eol-enforcement 772fd4e [Kousuke Saruta] Normized EOL character in make.bat and compute-classpath.cmd ac7f873 [Kousuke Saruta] Added an entry for .gitattributes to .rat-excludes 1570e77 [Kousuke Saruta] Added .gitattributes --- .gitattributes | 2 + .rat-excludes | 1 + bin/compute-classpath.cmd | 234 +++++++++--------- python/docs/make.bat | 12 +- python/docs/make2.bat | 486 +++++++++++++++++++------------------- 5 files changed, 369 insertions(+), 366 deletions(-) create mode 100644 .gitattributes diff --git a/.gitattributes b/.gitattributes new file mode 100644 index 0000000000000..2b65f6fe3cc80 --- /dev/null +++ b/.gitattributes @@ -0,0 +1,2 @@ +*.bat text eol=crlf +*.cmd text eol=crlf diff --git a/.rat-excludes b/.rat-excludes index ae9745673c87d..20e3372464386 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -1,5 +1,6 @@ target .gitignore +.gitattributes .project .classpath .mima-excludes diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 3cd0579aea8d3..a4c099fb45b14 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,117 +1,117 @@ -@echo off - -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" -rem script and the ExecutorRunner in standalone cluster mode. - -rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting -rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we -rem need to set it here because we use !datanucleus_jars! below. -if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion -setlocal enabledelayedexpansion -:skip_delayed_expansion - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" - -rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% - -if not "x%SPARK_CONF_DIR%"=="x" ( - set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% -) else ( - set CLASSPATH=%CLASSPATH%;%FWDIR%conf -) - -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) else ( - for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) - -set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% - -rem When Hive support is needed, Datanucleus jars must be included on the classpath. -rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. -rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is -rem built with Hive, so look for them there. -if exist "%FWDIR%RELEASE" ( - set datanucleus_dir=%FWDIR%lib -) else ( - set datanucleus_dir=%FWDIR%lib_managed\jars -) -set "datanucleus_jars=" -for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( - set datanucleus_jars=!datanucleus_jars!;%%d -) -set CLASSPATH=%CLASSPATH%;%datanucleus_jars% - -set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes - -set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes - -if "x%SPARK_TESTING%"=="x1" ( - rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH - rem so that local compilation takes precedence over assembled jar - set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% -) - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - -rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +rem script and the ExecutorRunner in standalone cluster mode. + +rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting +rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we +rem need to set it here because we use !datanucleus_jars! below. +if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion +setlocal enabledelayedexpansion +:skip_delayed_expansion + +set SCALA_VERSION=2.10 + +rem Figure out where the Spark framework is installed +set FWDIR=%~dp0..\ + +rem Load environment variables from conf\spark-env.cmd, if it exists +if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" + +rem Build up classpath +set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% + +if not "x%SPARK_CONF_DIR%"=="x" ( + set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% +) else ( + set CLASSPATH=%CLASSPATH%;%FWDIR%conf +) + +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) else ( + for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) + +set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% + +rem When Hive support is needed, Datanucleus jars must be included on the classpath. +rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is +rem built with Hive, so look for them there. +if exist "%FWDIR%RELEASE" ( + set datanucleus_dir=%FWDIR%lib +) else ( + set datanucleus_dir=%FWDIR%lib_managed\jars +) +set "datanucleus_jars=" +for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( + set datanucleus_jars=!datanucleus_jars!;%%d +) +set CLASSPATH=%CLASSPATH%;%datanucleus_jars% + +set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes + +set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes + +if "x%SPARK_TESTING%"=="x1" ( + rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH + rem so that local compilation takes precedence over assembled jar + set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% +) + +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + +rem A bit of a hack to allow calling this script within run2.cmd without seeing output +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/python/docs/make.bat b/python/docs/make.bat index c011e82b4a35a..cc29acdc19686 100644 --- a/python/docs/make.bat +++ b/python/docs/make.bat @@ -1,6 +1,6 @@ -@ECHO OFF - -rem This is the entry point for running Sphinx documentation. To avoid polluting the -rem environment, it just launches a new cmd to do the real work. - -cmd /V /E /C %~dp0make2.bat %* +@ECHO OFF + +rem This is the entry point for running Sphinx documentation. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. + +cmd /V /E /C %~dp0make2.bat %* diff --git a/python/docs/make2.bat b/python/docs/make2.bat index 7bcaeafad13d7..05d22eb5cdd23 100644 --- a/python/docs/make2.bat +++ b/python/docs/make2.bat @@ -1,243 +1,243 @@ -@ECHO OFF - -REM Command file for Sphinx documentation - - -if "%SPHINXBUILD%" == "" ( - set SPHINXBUILD=sphinx-build -) -set BUILDDIR=_build -set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% . -set I18NSPHINXOPTS=%SPHINXOPTS% . -if NOT "%PAPER%" == "" ( - set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% - set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% -) - -if "%1" == "" goto help - -if "%1" == "help" ( - :help - echo.Please use `make ^` where ^ is one of - echo. html to make standalone HTML files - echo. dirhtml to make HTML files named index.html in directories - echo. singlehtml to make a single large HTML file - echo. pickle to make pickle files - echo. json to make JSON files - echo. htmlhelp to make HTML files and a HTML help project - echo. qthelp to make HTML files and a qthelp project - echo. devhelp to make HTML files and a Devhelp project - echo. epub to make an epub - echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter - echo. text to make text files - echo. man to make manual pages - echo. texinfo to make Texinfo files - echo. gettext to make PO message catalogs - echo. changes to make an overview over all changed/added/deprecated items - echo. xml to make Docutils-native XML files - echo. pseudoxml to make pseudoxml-XML files for display purposes - echo. linkcheck to check all external links for integrity - echo. doctest to run all doctests embedded in the documentation if enabled - goto end -) - -if "%1" == "clean" ( - for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i - del /q /s %BUILDDIR%\* - goto end -) - - -%SPHINXBUILD% 2> nul -if errorlevel 9009 ( - echo. - echo.The 'sphinx-build' command was not found. Make sure you have Sphinx - echo.installed, then set the SPHINXBUILD environment variable to point - echo.to the full path of the 'sphinx-build' executable. Alternatively you - echo.may add the Sphinx directory to PATH. - echo. - echo.If you don't have Sphinx installed, grab it from - echo.http://sphinx-doc.org/ - exit /b 1 -) - -if "%1" == "html" ( - %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/html. - goto end -) - -if "%1" == "dirhtml" ( - %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. - goto end -) - -if "%1" == "singlehtml" ( - %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. - goto end -) - -if "%1" == "pickle" ( - %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can process the pickle files. - goto end -) - -if "%1" == "json" ( - %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can process the JSON files. - goto end -) - -if "%1" == "htmlhelp" ( - %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can run HTML Help Workshop with the ^ -.hhp project file in %BUILDDIR%/htmlhelp. - goto end -) - -if "%1" == "qthelp" ( - %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can run "qcollectiongenerator" with the ^ -.qhcp project file in %BUILDDIR%/qthelp, like this: - echo.^> qcollectiongenerator %BUILDDIR%\qthelp\pyspark.qhcp - echo.To view the help file: - echo.^> assistant -collectionFile %BUILDDIR%\qthelp\pyspark.ghc - goto end -) - -if "%1" == "devhelp" ( - %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. - goto end -) - -if "%1" == "epub" ( - %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The epub file is in %BUILDDIR%/epub. - goto end -) - -if "%1" == "latex" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "latexpdf" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - cd %BUILDDIR%/latex - make all-pdf - cd %BUILDDIR%/.. - echo. - echo.Build finished; the PDF files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "latexpdfja" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - cd %BUILDDIR%/latex - make all-pdf-ja - cd %BUILDDIR%/.. - echo. - echo.Build finished; the PDF files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "text" ( - %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The text files are in %BUILDDIR%/text. - goto end -) - -if "%1" == "man" ( - %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The manual pages are in %BUILDDIR%/man. - goto end -) - -if "%1" == "texinfo" ( - %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. - goto end -) - -if "%1" == "gettext" ( - %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The message catalogs are in %BUILDDIR%/locale. - goto end -) - -if "%1" == "changes" ( - %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes - if errorlevel 1 exit /b 1 - echo. - echo.The overview file is in %BUILDDIR%/changes. - goto end -) - -if "%1" == "linkcheck" ( - %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck - if errorlevel 1 exit /b 1 - echo. - echo.Link check complete; look for any errors in the above output ^ -or in %BUILDDIR%/linkcheck/output.txt. - goto end -) - -if "%1" == "doctest" ( - %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest - if errorlevel 1 exit /b 1 - echo. - echo.Testing of doctests in the sources finished, look at the ^ -results in %BUILDDIR%/doctest/output.txt. - goto end -) - -if "%1" == "xml" ( - %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The XML files are in %BUILDDIR%/xml. - goto end -) - -if "%1" == "pseudoxml" ( - %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. - goto end -) - -:end +@ECHO OFF + +REM Command file for Sphinx documentation + + +if "%SPHINXBUILD%" == "" ( + set SPHINXBUILD=sphinx-build +) +set BUILDDIR=_build +set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% . +set I18NSPHINXOPTS=%SPHINXOPTS% . +if NOT "%PAPER%" == "" ( + set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% + set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% +) + +if "%1" == "" goto help + +if "%1" == "help" ( + :help + echo.Please use `make ^` where ^ is one of + echo. html to make standalone HTML files + echo. dirhtml to make HTML files named index.html in directories + echo. singlehtml to make a single large HTML file + echo. pickle to make pickle files + echo. json to make JSON files + echo. htmlhelp to make HTML files and a HTML help project + echo. qthelp to make HTML files and a qthelp project + echo. devhelp to make HTML files and a Devhelp project + echo. epub to make an epub + echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter + echo. text to make text files + echo. man to make manual pages + echo. texinfo to make Texinfo files + echo. gettext to make PO message catalogs + echo. changes to make an overview over all changed/added/deprecated items + echo. xml to make Docutils-native XML files + echo. pseudoxml to make pseudoxml-XML files for display purposes + echo. linkcheck to check all external links for integrity + echo. doctest to run all doctests embedded in the documentation if enabled + goto end +) + +if "%1" == "clean" ( + for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i + del /q /s %BUILDDIR%\* + goto end +) + + +%SPHINXBUILD% 2> nul +if errorlevel 9009 ( + echo. + echo.The 'sphinx-build' command was not found. Make sure you have Sphinx + echo.installed, then set the SPHINXBUILD environment variable to point + echo.to the full path of the 'sphinx-build' executable. Alternatively you + echo.may add the Sphinx directory to PATH. + echo. + echo.If you don't have Sphinx installed, grab it from + echo.http://sphinx-doc.org/ + exit /b 1 +) + +if "%1" == "html" ( + %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/html. + goto end +) + +if "%1" == "dirhtml" ( + %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. + goto end +) + +if "%1" == "singlehtml" ( + %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. + goto end +) + +if "%1" == "pickle" ( + %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the pickle files. + goto end +) + +if "%1" == "json" ( + %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the JSON files. + goto end +) + +if "%1" == "htmlhelp" ( + %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run HTML Help Workshop with the ^ +.hhp project file in %BUILDDIR%/htmlhelp. + goto end +) + +if "%1" == "qthelp" ( + %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run "qcollectiongenerator" with the ^ +.qhcp project file in %BUILDDIR%/qthelp, like this: + echo.^> qcollectiongenerator %BUILDDIR%\qthelp\pyspark.qhcp + echo.To view the help file: + echo.^> assistant -collectionFile %BUILDDIR%\qthelp\pyspark.ghc + goto end +) + +if "%1" == "devhelp" ( + %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. + goto end +) + +if "%1" == "epub" ( + %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The epub file is in %BUILDDIR%/epub. + goto end +) + +if "%1" == "latex" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdf" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf + cd %BUILDDIR%/.. + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdfja" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf-ja + cd %BUILDDIR%/.. + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "text" ( + %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The text files are in %BUILDDIR%/text. + goto end +) + +if "%1" == "man" ( + %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The manual pages are in %BUILDDIR%/man. + goto end +) + +if "%1" == "texinfo" ( + %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. + goto end +) + +if "%1" == "gettext" ( + %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The message catalogs are in %BUILDDIR%/locale. + goto end +) + +if "%1" == "changes" ( + %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes + if errorlevel 1 exit /b 1 + echo. + echo.The overview file is in %BUILDDIR%/changes. + goto end +) + +if "%1" == "linkcheck" ( + %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck + if errorlevel 1 exit /b 1 + echo. + echo.Link check complete; look for any errors in the above output ^ +or in %BUILDDIR%/linkcheck/output.txt. + goto end +) + +if "%1" == "doctest" ( + %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest + if errorlevel 1 exit /b 1 + echo. + echo.Testing of doctests in the sources finished, look at the ^ +results in %BUILDDIR%/doctest/output.txt. + goto end +) + +if "%1" == "xml" ( + %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The XML files are in %BUILDDIR%/xml. + goto end +) + +if "%1" == "pseudoxml" ( + %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. + goto end +) + +:end From 087e31a7a0550f520c2e21b0ceaf36b969e050b4 Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Fri, 31 Oct 2014 14:36:55 -0700 Subject: [PATCH 114/115] [HOT FIX] Yarn stable tests don't compile This is caused by this commit: acd4ac7c9a503445e27739708cf36e19119b8ddc Author: andrewor14 Author: Andrew Or Closes #3041 from andrewor14/yarn-hot-fix and squashes the following commits: e5deba1 [andrewor14] Add new line at the end (minor) aa998e8 [Andrew Or] Compilation hot fix --- .../spark/deploy/yarn/YarnAllocator.scala | 30 +++++++++++-------- .../deploy/yarn/YarnAllocatorSuite.scala | 4 +-- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index e6196194acbb4..b32e15738f28b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -64,6 +64,8 @@ private[yarn] abstract class YarnAllocator( securityMgr: SecurityManager) extends Logging { + import YarnAllocator._ + // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. @@ -439,19 +441,6 @@ private[yarn] abstract class YarnAllocator( } } - private val MEM_REGEX = "[0-9.]+ [KMG]B" - private val PMEM_EXCEEDED_PATTERN = - Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used") - private val VMEM_EXCEEDED_PATTERN = - Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used") - - def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = { - val matcher = pattern.matcher(diagnostics) - val diag = if (matcher.find()) " " + matcher.group() + "." else "" - ("Container killed by YARN for exceeding memory limits." + diag - + " Consider boosting spark.yarn.executor.memoryOverhead.") - } - protected def allocatedContainersOnHost(host: String): Int = { var retval = 0 allocatedHostToContainersMap.synchronized { @@ -532,3 +521,18 @@ private[yarn] abstract class YarnAllocator( } } + +private object YarnAllocator { + val MEM_REGEX = "[0-9.]+ [KMG]B" + val PMEM_EXCEEDED_PATTERN = + Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used") + val VMEM_EXCEEDED_PATTERN = + Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used") + + def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = { + val matcher = pattern.matcher(diagnostics) + val diag = if (matcher.find()) " " + matcher.group() + "." else "" + ("Container killed by YARN for exceeding memory limits." + diag + + " Consider boosting spark.yarn.executor.memoryOverhead.") + } +} diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 9fff63fb25156..8d184a09d64cc 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import org.apache.spark.deploy.yarn.MemLimitLogger._ +import org.apache.spark.deploy.yarn.YarnAllocator._ import org.scalatest.FunSuite class YarnAllocatorSuite extends FunSuite { @@ -31,4 +31,4 @@ class YarnAllocatorSuite extends FunSuite { assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used.")) assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used.")) } -} \ No newline at end of file +} From 23f73f525ce3d2b4a614e60f4f9170c860ab93da Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 31 Oct 2014 17:22:52 -0700 Subject: [PATCH 115/115] SPARK-4175. Exception on stage page Author: Sandy Ryza Closes #3043 from sryza/sandy-spark-4175 and squashes the following commits: e327340 [Sandy Ryza] SPARK-4175. Exception on stage page --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index bf45272aefde4..7cc03b7d333df 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -312,7 +312,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { else metrics.map(_.executorRunTime).getOrElse(1L) val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") - val schedulerDelay = getSchedulerDelay(info, metrics.get) + val schedulerDelay = metrics.map(getSchedulerDelay(info, _)).getOrElse(0L) val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = info.gettingResultTime