diff --git a/core/pom.xml b/core/pom.xml
index 202678779150b1046c558e6ea2b74add396b9366..0e53a79fd2235cb94d66cf395af5412a9c4b405d 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -46,30 +46,10 @@
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>chill_${scala.binary.version}</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.ow2.asm</groupId>
-          <artifactId>asm</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.ow2.asm</groupId>
-          <artifactId>asm-commons</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>chill-java</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.ow2.asm</groupId>
-          <artifactId>asm</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.ow2.asm</groupId>
-          <artifactId>asm-commons</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
diff --git a/dev/run-tests.py b/dev/run-tests.py
index b6d181418f027f482bc5e87ec123a22cf1b1e0da..d1852b95bb2924b2fe4b4aeb4577818e98d89d13 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -273,6 +273,7 @@ def get_hadoop_profiles(hadoop_version):
         "hadoop2.0": ["-Phadoop-1", "-Dhadoop.version=2.0.0-mr1-cdh4.1.1"],
         "hadoop2.2": ["-Pyarn", "-Phadoop-2.2"],
         "hadoop2.3": ["-Pyarn", "-Phadoop-2.3", "-Dhadoop.version=2.3.0"],
+        "hadoop2.6": ["-Pyarn", "-Phadoop-2.6"],
     }
 
     if hadoop_version in sbt_maven_hadoop_profiles:
@@ -289,7 +290,7 @@ def build_spark_maven(hadoop_version):
     mvn_goals = ["clean", "package", "-DskipTests"]
     profiles_and_goals = build_profiles + mvn_goals
 
-    print("[info] Building Spark (w/Hive 0.13.1) using Maven with these arguments: ",
+    print("[info] Building Spark (w/Hive 1.2.1) using Maven with these arguments: ",
           " ".join(profiles_and_goals))
 
     exec_maven(profiles_and_goals)
@@ -305,14 +306,14 @@ def build_spark_sbt(hadoop_version):
                  "streaming-kinesis-asl-assembly/assembly"]
     profiles_and_goals = build_profiles + sbt_goals
 
-    print("[info] Building Spark (w/Hive 0.13.1) using SBT with these arguments: ",
+    print("[info] Building Spark (w/Hive 1.2.1) using SBT with these arguments: ",
           " ".join(profiles_and_goals))
 
     exec_sbt(profiles_and_goals)
 
 
 def build_apache_spark(build_tool, hadoop_version):
-    """Will build Spark against Hive v0.13.1 given the passed in build tool (either `sbt` or
+    """Will build Spark against Hive v1.2.1 given the passed in build tool (either `sbt` or
     `maven`). Defaults to using `sbt`."""
 
     set_title_and_block("Building Spark", "BLOCK_BUILD")
diff --git a/pom.xml b/pom.xml
index be0dac953abf7bec222146204bc9abf5884fd8bf..a958cec867eae3ce3391b8934642f71507225c3c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -134,11 +134,12 @@
     <curator.version>2.4.0</curator.version>
     <hive.group>org.spark-project.hive</hive.group>
     <!-- Version used in Maven Hive dependency -->
-    <hive.version>0.13.1a</hive.version>
+    <hive.version>1.2.1.spark</hive.version>
     <!-- Version used for internal directory structure -->
-    <hive.version.short>0.13.1</hive.version.short>
+    <hive.version.short>1.2.1</hive.version.short>
     <derby.version>10.10.1.1</derby.version>
     <parquet.version>1.7.0</parquet.version>
+    <hive.parquet.version>1.6.0</hive.parquet.version>
     <jblas.version>1.2.4</jblas.version>
     <jetty.version>8.1.14.v20131031</jetty.version>
     <orbit.version>3.0.0.v201112011016</orbit.version>
@@ -151,7 +152,10 @@
     <jets3t.version>0.7.1</jets3t.version>
     <aws.java.sdk.version>1.9.16</aws.java.sdk.version>
     <aws.kinesis.client.version>1.2.1</aws.kinesis.client.version>
+    <!--  org.apache.httpcomponents/httpclient-->
     <commons.httpclient.version>4.3.2</commons.httpclient.version>
+    <!--  commons-httpclient/commons-httpclient-->
+    <httpclient.classic.version>3.1</httpclient.classic.version>
     <commons.math3.version>3.4.1</commons.math3.version>
     <scala.version>2.10.4</scala.version>
     <scala.binary.version>2.10</scala.binary.version>
@@ -161,6 +165,23 @@
     <fasterxml.jackson.version>2.4.4</fasterxml.jackson.version>
     <snappy.version>1.1.1.7</snappy.version>
     <netlib.java.version>1.1.2</netlib.java.version>
+    <calcite.version>1.2.0-incubating</calcite.version>
+    <commons-codec.version>1.10</commons-codec.version>
+    <!-- org.apache.commons/commons-lang/-->
+    <commons-lang2.version>2.6</commons-lang2.version>
+    <!-- org.apache.commons/commons-lang3/-->
+    <commons-lang3.version>3.3.2</commons-lang3.version>
+    <datanucleus-core.version>3.2.10</datanucleus-core.version>
+    <janino.version>2.7.8</janino.version>
+    <jersey.version>1.9</jersey.version>
+    <joda.version>2.5</joda.version>
+    <jodd.version>3.5.2</jodd.version>
+    <jsr305.version>1.3.9</jsr305.version>
+    <libthrift.version>0.9.2</libthrift.version>
+
+    <!-- For maven shade plugin (see SPARK-8819) -->
+    <create.dependency.reduced.pom>false</create.dependency.reduced.pom>
+
     <test.java.home>${java.home}</test.java.home>
 
     <!--
@@ -188,7 +209,6 @@
     <MaxPermGen>512m</MaxPermGen>
     <CodeCacheSize>512m</CodeCacheSize>
   </properties>
-
   <repositories>
     <repository>
       <id>central</id>
@@ -246,6 +266,14 @@
         <enabled>false</enabled>
       </snapshots>
     </repository>
+    <repository>
+      <id>spark-hive-staging</id>
+      <name>Staging Repo for Hive 1.2.1 (Spark Version)</name>
+      <url>https://oss.sonatype.org/content/repositories/orgspark-project-1113</url>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+    </repository>
     <repository>
       <id>mapr-repo</id>
       <name>MapR Repository</name>
@@ -257,12 +285,13 @@
         <enabled>false</enabled>
       </snapshots>
     </repository>
+    <!-- returning unauthorized on some operations -->
     <repository>
       <id>spring-releases</id>
       <name>Spring Release Repository</name>
       <url>https://repo.spring.io/libs-release</url>
       <releases>
-        <enabled>true</enabled>
+        <enabled>false</enabled>
       </releases>
       <snapshots>
         <enabled>false</enabled>
@@ -402,12 +431,17 @@
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-lang3</artifactId>
-        <version>3.3.2</version>
+        <version>${commons-lang3.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.commons</groupId>
+        <artifactId>commons-lang</artifactId>
+        <version>${commons-lang2.version}</version>
       </dependency>
       <dependency>
         <groupId>commons-codec</groupId>
         <artifactId>commons-codec</artifactId>
-        <version>1.10</version>
+        <version>${commons-codec.version}</version>
       </dependency>
       <dependency>
         <groupId>org.apache.commons</groupId>
@@ -422,7 +456,12 @@
       <dependency>
         <groupId>com.google.code.findbugs</groupId>
         <artifactId>jsr305</artifactId>
-        <version>1.3.9</version>
+        <version>${jsr305.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>commons-httpclient</groupId>
+        <artifactId>commons-httpclient</artifactId>
+        <version>${httpclient.classic.version}</version>
       </dependency>
       <dependency>
         <groupId>org.apache.httpcomponents</groupId>
@@ -439,6 +478,16 @@
         <artifactId>selenium-java</artifactId>
         <version>2.42.2</version>
         <scope>test</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>io.netty</groupId>
+            <artifactId>netty</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <!-- Added for selenium only, and should match its dependent version: -->
       <dependency>
@@ -624,15 +673,26 @@
       <dependency>
         <groupId>com.sun.jersey</groupId>
         <artifactId>jersey-server</artifactId>
-        <version>1.9</version>
+        <version>${jersey.version}</version>
         <scope>${hadoop.deps.scope}</scope>
       </dependency>
       <dependency>
         <groupId>com.sun.jersey</groupId>
         <artifactId>jersey-core</artifactId>
-        <version>1.9</version>
+        <version>${jersey.version}</version>
         <scope>${hadoop.deps.scope}</scope>
       </dependency>
+      <dependency>
+        <groupId>com.sun.jersey</groupId>
+        <artifactId>jersey-json</artifactId>
+        <version>${jersey.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>stax</groupId>
+            <artifactId>stax-api</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
       <dependency>
         <groupId>org.scala-lang</groupId>
         <artifactId>scala-compiler</artifactId>
@@ -1022,58 +1082,499 @@
         <artifactId>hive-beeline</artifactId>
         <version>${hive.version}</version>
         <scope>${hive.deps.scope}</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-common</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-exec</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-jdbc</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-metastore</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-service</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-shims</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>${hive.group}</groupId>
         <artifactId>hive-cli</artifactId>
         <version>${hive.version}</version>
         <scope>${hive.deps.scope}</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-common</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-exec</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-jdbc</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-metastore</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-serde</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-service</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-shims</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>${hive.group}</groupId>
-        <artifactId>hive-exec</artifactId>
+        <artifactId>hive-common</artifactId>
         <version>${hive.version}</version>
         <scope>${hive.deps.scope}</scope>
         <exclusions>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-shims</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.ant</groupId>
+            <artifactId>ant</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
           <exclusion>
             <groupId>commons-logging</groupId>
             <artifactId>commons-logging</artifactId>
           </exclusion>
+        </exclusions>
+      </dependency>
+
+      <dependency>
+        <groupId>${hive.group}</groupId>
+        <artifactId>hive-exec</artifactId>
+<!--
+        <classifier>core</classifier>
+-->
+        <version>${hive.version}</version>
+        <scope>${hive.deps.scope}</scope>
+        <exclusions>
+
+          <!-- pull this in when needed; the explicit definition culls the surplis-->
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-metastore</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-shims</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-ant</artifactId>
+          </exclusion>
+          <!-- break the loop -->
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>spark-client</artifactId>
+          </exclusion>
+
+          <!-- excluded dependencies & transitive.
+           Some may be needed to be explicitly included-->
+          <exclusion>
+            <groupId>ant</groupId>
+            <artifactId>ant</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.ant</groupId>
+            <artifactId>ant</artifactId>
+          </exclusion>
           <exclusion>
             <groupId>com.esotericsoftware.kryo</groupId>
             <artifactId>kryo</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-httpclient</groupId>
+            <artifactId>commons-httpclient</artifactId>
+          </exclusion>
           <exclusion>
             <groupId>org.apache.avro</groupId>
             <artifactId>avro-mapred</artifactId>
           </exclusion>
+          <!--  this is needed and must be explicitly included later-->
+          <exclusion>
+            <groupId>org.apache.calcite</groupId>
+            <artifactId>calcite-core</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>apache-curator</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-client</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libfb303</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
       <dependency>
         <groupId>${hive.group}</groupId>
         <artifactId>hive-jdbc</artifactId>
         <version>${hive.version}</version>
-        <scope>${hive.deps.scope}</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-common</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-common</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-metastore</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-serde</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-service</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-shims</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpcore</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libfb303</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
+
       <dependency>
         <groupId>${hive.group}</groupId>
         <artifactId>hive-metastore</artifactId>
         <version>${hive.version}</version>
         <scope>${hive.deps.scope}</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-serde</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-shims</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libfb303</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
+
       <dependency>
         <groupId>${hive.group}</groupId>
         <artifactId>hive-serde</artifactId>
         <version>${hive.version}</version>
         <scope>${hive.deps.scope}</scope>
         <exclusions>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-common</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-shims</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libfb303</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
           <exclusion>
             <groupId>commons-logging</groupId>
             <artifactId>commons-logging</artifactId>
           </exclusion>
+        </exclusions>
+      </dependency>
+
+      <dependency>
+        <groupId>${hive.group}</groupId>
+        <artifactId>hive-service</artifactId>
+        <version>${hive.version}</version>
+        <scope>${hive.deps.scope}</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-common</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-exec</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-metastore</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>${hive.group}</groupId>
+            <artifactId>hive-shims</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-recipes</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libfb303</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+
+      <!-- hive shims pulls in hive 0.23 and a transitive dependency of the Hadoop version
+        Hive was built against. This dependency cuts out the YARN/hadoop dependency, which
+        is needed by Hive to submit work to a YARN cluster.-->
+      <dependency>
+        <groupId>${hive.group}</groupId>
+        <artifactId>hive-shims</artifactId>
+        <version>${hive.version}</version>
+        <scope>${hive.deps.scope}</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
           <exclusion>
             <groupId>commons-logging</groupId>
-            <artifactId>commons-logging-api</artifactId>
+            <artifactId>commons-logging</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
@@ -1095,6 +1596,12 @@
         <version>${parquet.version}</version>
         <scope>${parquet.test.deps.scope}</scope>
       </dependency>
+      <dependency>
+        <groupId>com.twitter</groupId>
+        <artifactId>parquet-hadoop-bundle</artifactId>
+        <version>${hive.parquet.version}</version>
+        <scope>runtime</scope>
+      </dependency>
       <dependency>
         <groupId>org.apache.flume</groupId>
         <artifactId>flume-ng-core</artifactId>
@@ -1135,6 +1642,125 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>org.apache.calcite</groupId>
+        <artifactId>calcite-core</artifactId>
+        <version>${calcite.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.codehaus.janino</groupId>
+            <artifactId>janino</artifactId>
+          </exclusion>
+          <!-- hsqldb interferes with the use of derby as the default db
+            in hive's use of datanucleus.
+          -->
+          <exclusion>
+            <groupId>org.hsqldb</groupId>
+            <artifactId>hsqldb</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.pentaho</groupId>
+            <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.calcite</groupId>
+        <artifactId>calcite-avatica</artifactId>
+        <version>${calcite.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.codehaus.janino</groupId>
+        <artifactId>janino</artifactId>
+        <version>${janino.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>joda-time</groupId>
+        <artifactId>joda-time</artifactId>
+        <version>${joda.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.jodd</groupId>
+        <artifactId>jodd-core</artifactId>
+        <version>${jodd.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.datanucleus</groupId>
+        <artifactId>datanucleus-core</artifactId>
+        <version>${datanucleus-core.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.thrift</groupId>
+        <artifactId>libthrift</artifactId>
+        <version>${libthrift.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpcore</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.thrift</groupId>
+        <artifactId>libfb303</artifactId>
+        <version>${libthrift.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpcore</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
@@ -1271,6 +1897,8 @@
               <spark.ui.showConsoleProgress>false</spark.ui.showConsoleProgress>
               <spark.driver.allowMultipleContexts>true</spark.driver.allowMultipleContexts>
               <spark.unsafe.exceptionOnMemoryLeak>true</spark.unsafe.exceptionOnMemoryLeak>
+              <!-- Needed by sql/hive tests. -->
+              <test.src.tables>src</test.src.tables>
             </systemProperties>
             <failIfNoTests>false</failIfNoTests>
           </configuration>
@@ -1305,6 +1933,8 @@
               <spark.ui.showConsoleProgress>false</spark.ui.showConsoleProgress>
               <spark.driver.allowMultipleContexts>true</spark.driver.allowMultipleContexts>
               <spark.unsafe.exceptionOnMemoryLeak>true</spark.unsafe.exceptionOnMemoryLeak>
+              <!-- Needed by sql/hive tests. -->
+              <test.src.tables>__not_used__</test.src.tables>
             </systemProperties>
           </configuration>
           <executions>
diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh
index de762acc8fa0e4fc3449e249e7e7f92b338cc5c1..0fbe795822fbf8d111890362db5d790042a03557 100755
--- a/sbin/spark-daemon.sh
+++ b/sbin/spark-daemon.sh
@@ -29,7 +29,7 @@
 #   SPARK_NICENESS The scheduling priority for daemons. Defaults to 0.
 ##
 
-usage="Usage: spark-daemon.sh [--config <conf-dir>] (start|stop|status) <spark-command> <spark-instance-number> <args...>"
+usage="Usage: spark-daemon.sh [--config <conf-dir>] (start|stop|submit|status) <spark-command> <spark-instance-number> <args...>"
 
 # if no args specified, show usage
 if [ $# -le 1 ]; then
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index f4b1cc3a4ffe7ecc4eba9993aa09cb799ca0fabd..75ab575dfde835cee4c7c506582f2aae77c2c3c8 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -66,7 +66,6 @@
     <dependency>
       <groupId>org.codehaus.janino</groupId>
       <artifactId>janino</artifactId>
-      <version>2.7.8</version>
     </dependency>
   </dependencies>
   <build>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala
index b4cdfd9e98f6f47c0e62773a835909370e78cec5..57478931cd509c566bc1038a8e7a93eca14f7237 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala
@@ -31,6 +31,14 @@ import org.apache.spark.util.Utils
 abstract class ParquetCompatibilityTest extends QueryTest with ParquetTest with BeforeAndAfterAll {
   protected var parquetStore: File = _
 
+  /**
+   * Optional path to a staging subdirectory which may be created during query processing
+   * (Hive does this).
+   * Parquet files under this directory will be ignored in [[readParquetSchema()]]
+   * @return an optional staging directory to ignore when scanning for parquet files.
+   */
+  protected def stagingDir: Option[String] = None
+
   override protected def beforeAll(): Unit = {
     parquetStore = Utils.createTempDir(namePrefix = "parquet-compat_")
     parquetStore.delete()
@@ -43,7 +51,10 @@ abstract class ParquetCompatibilityTest extends QueryTest with ParquetTest with
   def readParquetSchema(path: String): MessageType = {
     val fsPath = new Path(path)
     val fs = fsPath.getFileSystem(configuration)
-    val parquetFiles = fs.listStatus(fsPath).toSeq.filterNot(_.getPath.getName.startsWith("_"))
+    val parquetFiles = fs.listStatus(fsPath).toSeq.filterNot { status =>
+      status.getPath.getName.startsWith("_") ||
+        stagingDir.map(status.getPath.getName.startsWith).getOrElse(false)
+    }
     val footers = ParquetFileReader.readAllFootersInParallel(configuration, parquetFiles, true)
     footers.head.getParquetMetadata.getFileMetaData.getSchema
   }
diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml
index 73e6ccdb1eaf8e04229acb2f905281535df6a03f..2dfbcb2425a3721889da5c0fe370bac109e78375 100644
--- a/sql/hive-thriftserver/pom.xml
+++ b/sql/hive-thriftserver/pom.xml
@@ -60,21 +60,31 @@
       <groupId>${hive.group}</groupId>
       <artifactId>hive-jdbc</artifactId>
     </dependency>
+    <dependency>
+      <groupId>${hive.group}</groupId>
+      <artifactId>hive-service</artifactId>
+    </dependency>
     <dependency>
       <groupId>${hive.group}</groupId>
       <artifactId>hive-beeline</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-json</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-server</artifactId>
+    </dependency>
     <!-- Added for selenium: -->
     <dependency>
       <groupId>org.seleniumhq.selenium</groupId>
       <artifactId>selenium-java</artifactId>
       <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>io.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
   </dependencies>
   <build>
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/hive/service/server/HiveServerServerOptionsProcessor.scala b/sql/hive-thriftserver/src/main/scala/org/apache/hive/service/server/HiveServerServerOptionsProcessor.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2228f651e2387503835460309692aff688424df0
--- /dev/null
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/hive/service/server/HiveServerServerOptionsProcessor.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.hive.service.server
+
+import org.apache.hive.service.server.HiveServer2.{StartOptionExecutor, ServerOptionsProcessor}
+
+/**
+ * Class to upgrade a package-private class to public, and
+ * implement a `process()` operation consistent with
+ * the behavior of older Hive versions
+ * @param serverName name of the hive server
+ */
+private[apache] class HiveServerServerOptionsProcessor(serverName: String)
+    extends ServerOptionsProcessor(serverName) {
+
+  def process(args: Array[String]): Boolean = {
+    // A parse failure automatically triggers a system exit
+    val response = super.parse(args)
+    val executor = response.getServerOptionsExecutor()
+    // return true if the parsed option was to start the service
+    executor.isInstanceOf[StartOptionExecutor]
+  }
+}
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
index b7db80d93f852466e63e58c3ff972a87cb97b302..9c047347cb58d8958abf53862e982192daaf0a9a 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala
@@ -17,6 +17,9 @@
 
 package org.apache.spark.sql.hive.thriftserver
 
+import java.util.Locale
+import java.util.concurrent.atomic.AtomicBoolean
+
 import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 
@@ -24,7 +27,7 @@ import org.apache.commons.logging.LogFactory
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars
 import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService}
-import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor}
+import org.apache.hive.service.server.{HiveServerServerOptionsProcessor, HiveServer2}
 
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart}
@@ -65,7 +68,7 @@ object HiveThriftServer2 extends Logging {
   }
 
   def main(args: Array[String]) {
-    val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2")
+    val optionsProcessor = new HiveServerServerOptionsProcessor("HiveThriftServer2")
     if (!optionsProcessor.process(args)) {
       System.exit(-1)
     }
@@ -241,9 +244,12 @@ object HiveThriftServer2 extends Logging {
 private[hive] class HiveThriftServer2(hiveContext: HiveContext)
   extends HiveServer2
   with ReflectedCompositeService {
+  // state is tracked internally so that the server only attempts to shut down if it successfully
+  // started, and then once only.
+  private val started = new AtomicBoolean(false)
 
   override def init(hiveConf: HiveConf) {
-    val sparkSqlCliService = new SparkSQLCLIService(hiveContext)
+    val sparkSqlCliService = new SparkSQLCLIService(this, hiveContext)
     setSuperField(this, "cliService", sparkSqlCliService)
     addService(sparkSqlCliService)
 
@@ -259,8 +265,19 @@ private[hive] class HiveThriftServer2(hiveContext: HiveContext)
   }
 
   private def isHTTPTransportMode(hiveConf: HiveConf): Boolean = {
-    val transportMode: String = hiveConf.getVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE)
-    transportMode.equalsIgnoreCase("http")
+    val transportMode = hiveConf.getVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE)
+    transportMode.toLowerCase(Locale.ENGLISH).equals("http")
+  }
+
+
+  override def start(): Unit = {
+    super.start()
+    started.set(true)
   }
 
+  override def stop(): Unit = {
+    if (started.getAndSet(false)) {
+       super.stop()
+    }
+  }
 }
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index e8758887ff3a22c52b8d60d2bc6be959d8f61b45..833bf62d47d071bbe69ba6114346e0a3814a8182 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -32,8 +32,7 @@ import org.apache.hive.service.cli._
 import org.apache.hadoop.hive.ql.metadata.Hive
 import org.apache.hadoop.hive.ql.metadata.HiveException
 import org.apache.hadoop.hive.ql.session.SessionState
-import org.apache.hadoop.hive.shims.ShimLoader
-import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.hive.shims.Utils
 import org.apache.hive.service.cli.operation.ExecuteStatementOperation
 import org.apache.hive.service.cli.session.HiveSession
 
@@ -146,7 +145,7 @@ private[hive] class SparkExecuteStatementOperation(
     } else {
       val parentSessionState = SessionState.get()
       val hiveConf = getConfigForOperation()
-      val sparkServiceUGI = ShimLoader.getHadoopShims.getUGIForConf(hiveConf)
+      val sparkServiceUGI = Utils.getUGI()
       val sessionHive = getCurrentHive()
       val currentSqlSession = hiveContext.currentSession
 
@@ -174,7 +173,7 @@ private[hive] class SparkExecuteStatementOperation(
           }
 
           try {
-            ShimLoader.getHadoopShims().doAs(sparkServiceUGI, doAsAction)
+            sparkServiceUGI.doAs(doAsAction)
           } catch {
             case e: Exception =>
               setOperationException(new HiveSQLException(e))
@@ -201,7 +200,7 @@ private[hive] class SparkExecuteStatementOperation(
     }
   }
 
-  private def runInternal(): Unit = {
+  override def runInternal(): Unit = {
     statementId = UUID.randomUUID().toString
     logInfo(s"Running query '$statement' with $statementId")
     setState(OperationState.RUNNING)
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 f66a17b20915f376347da582c0b82a7390a1b266..d3886142b388d446791682bec29fa8b8b7841fa3 100644
--- 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
@@ -20,9 +20,10 @@ package org.apache.spark.sql.hive.thriftserver
 import scala.collection.JavaConversions._
 
 import java.io._
-import java.util.{ArrayList => JArrayList}
+import java.util.{ArrayList => JArrayList, Locale}
 
-import jline.{ConsoleReader, History}
+import jline.console.ConsoleReader
+import jline.console.history.FileHistory
 
 import org.apache.commons.lang3.StringUtils
 import org.apache.commons.logging.LogFactory
@@ -40,6 +41,10 @@ import org.apache.spark.Logging
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.util.Utils
 
+/**
+ * This code doesn't support remote connections in Hive 1.2+, as the underlying CliDriver
+ * has dropped its support.
+ */
 private[hive] object SparkSQLCLIDriver extends Logging {
   private var prompt = "spark-sql"
   private var continuedPrompt = "".padTo(prompt.length, ' ')
@@ -111,16 +116,9 @@ private[hive] object SparkSQLCLIDriver extends Logging {
     // Clean up after we exit
     Utils.addShutdownHook { () => SparkSQLEnv.stop() }
 
+    val remoteMode = isRemoteMode(sessionState)
     // "-h" option has been passed, so connect to Hive thrift server.
-    if (sessionState.getHost != null) {
-      sessionState.connect()
-      if (sessionState.isRemoteMode) {
-        prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + prompt
-        continuedPrompt = "".padTo(prompt.length, ' ')
-      }
-    }
-
-    if (!sessionState.isRemoteMode) {
+    if (!remoteMode) {
       // Hadoop-20 and above - we need to augment classpath using hiveconf
       // components.
       // See also: code in ExecDriver.java
@@ -131,6 +129,9 @@ private[hive] object SparkSQLCLIDriver extends Logging {
       }
       conf.setClassLoader(loader)
       Thread.currentThread().setContextClassLoader(loader)
+    } else {
+      // Hive 1.2 + not supported in CLI
+      throw new RuntimeException("Remote operations not supported")
     }
 
     val cli = new SparkSQLCLIDriver
@@ -171,14 +172,14 @@ private[hive] object SparkSQLCLIDriver extends Logging {
     val reader = new ConsoleReader()
     reader.setBellEnabled(false)
     // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true)))
-    CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e))
+    CliDriver.getCommandCompleter.foreach((e) => reader.addCompleter(e))
 
     val historyDirectory = System.getProperty("user.home")
 
     try {
       if (new File(historyDirectory).exists()) {
         val historyFile = historyDirectory + File.separator + ".hivehistory"
-        reader.setHistory(new History(new File(historyFile)))
+        reader.setHistory(new FileHistory(new File(historyFile)))
       } else {
         logWarning("WARNING: Directory for Hive history file: " + historyDirectory +
                            " does not exist.   History will not be available during this session.")
@@ -190,10 +191,14 @@ private[hive] object SparkSQLCLIDriver extends Logging {
         logWarning(e.getMessage)
     }
 
+    // TODO: missing
+/*
     val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport")
     clientTransportTSocketField.setAccessible(true)
 
     transport = clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket]
+*/
+    transport = null
 
     var ret = 0
     var prefix = ""
@@ -230,6 +235,13 @@ private[hive] object SparkSQLCLIDriver extends Logging {
 
     System.exit(ret)
   }
+
+
+  def isRemoteMode(state: CliSessionState): Boolean = {
+    //    sessionState.isRemoteMode
+    state.isHiveServerQuery
+  }
+
 }
 
 private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
@@ -239,25 +251,33 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
 
   private val console = new SessionState.LogHelper(LOG)
 
+  private val isRemoteMode = {
+    SparkSQLCLIDriver.isRemoteMode(sessionState)
+  }
+
   private val conf: Configuration =
     if (sessionState != null) sessionState.getConf else new Configuration()
 
   // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver
   // because the Hive unit tests do not go through the main() code path.
-  if (!sessionState.isRemoteMode) {
+  if (!isRemoteMode) {
     SparkSQLEnv.init()
+  } else {
+    // Hive 1.2 + not supported in CLI
+    throw new RuntimeException("Remote operations not supported")
   }
 
   override def processCmd(cmd: String): Int = {
     val cmd_trimmed: String = cmd.trim()
+    val cmd_lower = cmd_trimmed.toLowerCase(Locale.ENGLISH)
     val tokens: Array[String] = cmd_trimmed.split("\\s+")
     val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim()
-    if (cmd_trimmed.toLowerCase.equals("quit") ||
-      cmd_trimmed.toLowerCase.equals("exit") ||
-      tokens(0).equalsIgnoreCase("source") ||
+    if (cmd_lower.equals("quit") ||
+      cmd_lower.equals("exit") ||
+      tokens(0).toLowerCase(Locale.ENGLISH).equals("source") ||
       cmd_trimmed.startsWith("!") ||
       tokens(0).toLowerCase.equals("list") ||
-      sessionState.isRemoteMode) {
+      isRemoteMode) {
       val start = System.currentTimeMillis()
       super.processCmd(cmd)
       val end = System.currentTimeMillis()
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 41f647d5f8c5a45c8f4b6dd0dfdc45467438b025..644165acf70a727fab61375a5168597dbff87e07 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
@@ -23,11 +23,12 @@ import javax.security.auth.login.LoginException
 
 import org.apache.commons.logging.Log
 import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.shims.ShimLoader
+import org.apache.hadoop.hive.shims.Utils
 import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hive.service.Service.STATE
 import org.apache.hive.service.auth.HiveAuthFactory
 import org.apache.hive.service.cli._
+import org.apache.hive.service.server.HiveServer2
 import org.apache.hive.service.{AbstractService, Service, ServiceException}
 
 import org.apache.spark.sql.hive.HiveContext
@@ -35,22 +36,22 @@ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
 
 import scala.collection.JavaConversions._
 
-private[hive] class SparkSQLCLIService(hiveContext: HiveContext)
-  extends CLIService
+private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, hiveContext: HiveContext)
+  extends CLIService(hiveServer)
   with ReflectedCompositeService {
 
   override def init(hiveConf: HiveConf) {
     setSuperField(this, "hiveConf", hiveConf)
 
-    val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext)
+    val sparkSqlSessionManager = new SparkSQLSessionManager(hiveServer, hiveContext)
     setSuperField(this, "sessionManager", sparkSqlSessionManager)
     addService(sparkSqlSessionManager)
     var sparkServiceUGI: UserGroupInformation = null
 
-    if (ShimLoader.getHadoopShims.isSecurityEnabled) {
+    if (UserGroupInformation.isSecurityEnabled) {
       try {
         HiveAuthFactory.loginFromKeytab(hiveConf)
-        sparkServiceUGI = ShimLoader.getHadoopShims.getUGIForConf(hiveConf)
+        sparkServiceUGI = Utils.getUGI()
         setSuperField(this, "serviceUGI", sparkServiceUGI)
       } catch {
         case e @ (_: IOException | _: LoginException) =>
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala
index 2d5ee68002286ae5c2d784ec48f805a0606c5002..92ac0ec3fca298847507143f23fab38d3d590f05 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala
@@ -25,14 +25,15 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars
 import org.apache.hive.service.cli.SessionHandle
 import org.apache.hive.service.cli.session.SessionManager
 import org.apache.hive.service.cli.thrift.TProtocolVersion
+import org.apache.hive.service.server.HiveServer2
 
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._
 import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager
 
 
-private[hive] class SparkSQLSessionManager(hiveContext: HiveContext)
-  extends SessionManager
+private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, hiveContext: HiveContext)
+  extends SessionManager(hiveServer)
   with ReflectedCompositeService {
 
   private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext)
@@ -55,12 +56,14 @@ private[hive] class SparkSQLSessionManager(hiveContext: HiveContext)
       protocol: TProtocolVersion,
       username: String,
       passwd: String,
+      ipAddress: String,
       sessionConf: java.util.Map[String, String],
       withImpersonation: Boolean,
       delegationToken: String): SessionHandle = {
     hiveContext.openSession()
-    val sessionHandle = super.openSession(
-      protocol, username, passwd, sessionConf, withImpersonation, delegationToken)
+    val sessionHandle =
+      super.openSession(protocol, username, passwd, ipAddress, sessionConf, withImpersonation,
+          delegationToken)
     val session = super.getSession(sessionHandle)
     HiveThriftServer2.listener.onSessionCreated(
       session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername)
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
index df80d04b4080121c86db79542ec04e31dacd674b..121b3e077f71fcd2f28b24d3e3928166f97e0ce9 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
@@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer
 import scala.concurrent.duration._
 import scala.concurrent.{Await, Promise}
 import scala.sys.process.{Process, ProcessLogger}
+import scala.util.Failure
 
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars
 import org.scalatest.BeforeAndAfter
@@ -37,31 +38,46 @@ import org.apache.spark.util.Utils
 class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging {
   val warehousePath = Utils.createTempDir()
   val metastorePath = Utils.createTempDir()
+  val scratchDirPath = Utils.createTempDir()
 
   before {
-      warehousePath.delete()
-      metastorePath.delete()
+    warehousePath.delete()
+    metastorePath.delete()
+    scratchDirPath.delete()
   }
 
   after {
-      warehousePath.delete()
-      metastorePath.delete()
+    warehousePath.delete()
+    metastorePath.delete()
+    scratchDirPath.delete()
   }
 
+  /**
+   * Run a CLI operation and expect all the queries and expected answers to be returned.
+   * @param timeout maximum time for the commands to complete
+   * @param extraArgs any extra arguments
+   * @param errorResponses a sequence of strings whose presence in the stdout of the forked process
+   *                       is taken as an immediate error condition. That is: if a line beginning
+   *                       with one of these strings is found, fail the test immediately.
+   *                       The default value is `Seq("Error:")`
+   *
+   * @param queriesAndExpectedAnswers one or more tupes of query + answer
+   */
   def runCliWithin(
       timeout: FiniteDuration,
-      extraArgs: Seq[String] = Seq.empty)(
+      extraArgs: Seq[String] = Seq.empty,
+      errorResponses: Seq[String] = Seq("Error:"))(
       queriesAndExpectedAnswers: (String, String)*): Unit = {
 
     val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip
-    val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator)
-
     val command = {
+      val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator)
       val jdbcUrl = s"jdbc:derby:;databaseName=$metastorePath;create=true"
       s"""$cliScript
          |  --master local
          |  --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl
          |  --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath
+         |  --hiveconf ${ConfVars.SCRATCHDIR}=$scratchDirPath
        """.stripMargin.split("\\s+").toSeq ++ extraArgs
     }
 
@@ -81,6 +97,12 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging {
         if (next == expectedAnswers.size) {
           foundAllExpectedAnswers.trySuccess(())
         }
+      } else {
+        errorResponses.foreach( r => {
+          if (line.startsWith(r)) {
+            foundAllExpectedAnswers.tryFailure(
+              new RuntimeException(s"Failed with error line '$line'"))
+          }})
       }
     }
 
@@ -88,16 +110,44 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging {
     val process = (Process(command, None) #< queryStream).run(
       ProcessLogger(captureOutput("stdout"), captureOutput("stderr")))
 
+    // catch the output value
+    class exitCodeCatcher extends Runnable {
+      var exitValue = 0
+
+      override def run(): Unit = {
+        try {
+          exitValue = process.exitValue()
+        } catch {
+          case rte: RuntimeException =>
+            // ignored as it will get triggered when the process gets destroyed
+            logDebug("Ignoring exception while waiting for exit code", rte)
+        }
+        if (exitValue != 0) {
+          // process exited: fail fast
+          foundAllExpectedAnswers.tryFailure(
+            new RuntimeException(s"Failed with exit code $exitValue"))
+        }
+      }
+    }
+    // spin off the code catche thread. No attempt is made to kill this
+    // as it will exit once the launched process terminates.
+    val codeCatcherThread = new Thread(new exitCodeCatcher())
+    codeCatcherThread.start()
+
     try {
-      Await.result(foundAllExpectedAnswers.future, timeout)
+      Await.ready(foundAllExpectedAnswers.future, timeout)
+      foundAllExpectedAnswers.future.value match {
+        case Some(Failure(t)) => throw t
+        case _ =>
+      }
     } catch { case cause: Throwable =>
-      logError(
+      val message =
         s"""
            |=======================
            |CliSuite failure output
            |=======================
            |Spark SQL CLI command line: ${command.mkString(" ")}
-           |
+           |Exception: $cause
            |Executed query $next "${queries(next)}",
            |But failed to capture expected output "${expectedAnswers(next)}" within $timeout.
            |
@@ -105,8 +155,9 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging {
            |===========================
            |End CliSuite failure output
            |===========================
-         """.stripMargin, cause)
-      throw cause
+         """.stripMargin
+      logError(message, cause)
+      fail(message, cause)
     } finally {
       process.destroy()
     }
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index 39b31523e07cb763a8de80e8cc06a0d2e9ad38f6..8374629b5d45a96be900bb50c2860b0d77f8d43d 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.thriftserver
 
 import java.io.File
 import java.net.URL
-import java.nio.charset.StandardCharsets
 import java.sql.{Date, DriverManager, SQLException, Statement}
 
 import scala.collection.mutable.ArrayBuffer
@@ -492,7 +491,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl
         new File(s"$tempLog4jConf/log4j.properties"),
         UTF_8)
 
-      tempLog4jConf + File.pathSeparator + sys.props("java.class.path")
+      tempLog4jConf // + File.pathSeparator + sys.props("java.class.path")
     }
 
     s"""$startScript
@@ -508,6 +507,20 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl
      """.stripMargin.split("\\s+").toSeq
   }
 
+  /**
+   * String to scan for when looking for the the thrift binary endpoint running.
+   * This can change across Hive versions.
+   */
+  val THRIFT_BINARY_SERVICE_LIVE = "Starting ThriftBinaryCLIService on port"
+
+  /**
+   * String to scan for when looking for the the thrift HTTP endpoint running.
+   * This can change across Hive versions.
+   */
+  val THRIFT_HTTP_SERVICE_LIVE = "Started ThriftHttpCLIService in http"
+
+  val SERVER_STARTUP_TIMEOUT = 1.minute
+
   private def startThriftServer(port: Int, attempt: Int) = {
     warehousePath = Utils.createTempDir()
     warehousePath.delete()
@@ -545,23 +558,26 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl
 
     // Ensures that the following "tail" command won't fail.
     logPath.createNewFile()
+    val successLines = Seq(THRIFT_BINARY_SERVICE_LIVE, THRIFT_HTTP_SERVICE_LIVE)
+    val failureLines = Seq("HiveServer2 is stopped", "Exception in thread", "Error:")
     logTailingProcess =
       // Using "-n +0" to make sure all lines in the log file are checked.
       Process(s"/usr/bin/env tail -n +0 -f ${logPath.getCanonicalPath}").run(ProcessLogger(
         (line: String) => {
           diagnosisBuffer += line
-
-          if (line.contains("ThriftBinaryCLIService listening on") ||
-              line.contains("Started ThriftHttpCLIService in http")) {
-            serverStarted.trySuccess(())
-          } else if (line.contains("HiveServer2 is stopped")) {
-            // This log line appears when the server fails to start and terminates gracefully (e.g.
-            // because of port contention).
-            serverStarted.tryFailure(new RuntimeException("Failed to start HiveThriftServer2"))
-          }
+          successLines.foreach(r => {
+            if (line.contains(r)) {
+              serverStarted.trySuccess(())
+            }
+          })
+          failureLines.foreach(r => {
+            if (line.contains(r)) {
+              serverStarted.tryFailure(new RuntimeException(s"Failed with output '$line'"))
+            }
+          })
         }))
 
-    Await.result(serverStarted.future, 2.minute)
+    Await.result(serverStarted.future, SERVER_STARTUP_TIMEOUT)
   }
 
   private def stopThriftServer(): Unit = {
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 53d5b22b527b2f72212d3a77452d3b82bb87b3b4..c46a4a4b0be548efdb911a05ca55f5161d6a3d3c 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
@@ -267,7 +267,34 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "date_udf",
 
     // Unlike Hive, we do support log base in (0, 1.0], therefore disable this
-    "udf7"
+    "udf7",
+
+    // Trivial changes to DDL output
+    "compute_stats_empty_table",
+    "compute_stats_long",
+    "create_view_translate",
+    "show_create_table_serde",
+    "show_tblproperties",
+
+    // Odd changes to output
+    "merge4",
+
+    // Thift is broken...
+    "inputddl8",
+
+    // Hive changed ordering of ddl:
+    "varchar_union1",
+
+    // Parser changes in Hive 1.2
+    "input25",
+    "input26",
+
+    // Uses invalid table name
+    "innerjoin",
+
+    // classpath problems
+    "compute_stats.*",
+    "udf_bitmap_.*"
   )
 
   /**
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index b00f320318be090f015493da2386476efb86111e..be1607476e25440147e4901bedf82c74d2cfbebf 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -36,6 +36,11 @@
   </properties>
 
   <dependencies>
+    <!-- Added for Hive Parquet SerDe -->
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>parquet-hadoop-bundle</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-core_${scala.binary.version}</artifactId>
@@ -53,32 +58,42 @@
       <artifactId>spark-sql_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+<!--
     <dependency>
-      <groupId>${hive.group}</groupId>
-      <artifactId>hive-metastore</artifactId>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
     </dependency>
     <dependency>
-      <groupId>commons-httpclient</groupId>
-      <artifactId>commons-httpclient</artifactId>
-      <version>3.1</version>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${protobuf.version}</version>
     </dependency>
     <dependency>
       <groupId>${hive.group}</groupId>
-      <artifactId>hive-exec</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpclient</artifactId>
-      <version>${commons.httpclient.version}</version>
+      <artifactId>hive-common</artifactId>
     </dependency>
+-->
     <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
+      <groupId>${hive.group}</groupId>
+      <artifactId>hive-exec</artifactId>
+<!--
+      <classifier>core</classifier>
+-->
     </dependency>
     <dependency>
       <groupId>${hive.group}</groupId>
-      <artifactId>hive-serde</artifactId>
+      <artifactId>hive-metastore</artifactId>
     </dependency>
+    <!--
+        <dependency>
+          <groupId>${hive.group}</groupId>
+          <artifactId>hive-serde</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>${hive.group}</groupId>
+          <artifactId>hive-shims</artifactId>
+        </dependency>
+    -->
     <!-- hive-serde already depends on avro, but this brings in customized config of avro deps from parent -->
     <dependency>
       <groupId>org.apache.avro</groupId>
@@ -91,6 +106,55 @@
       <artifactId>avro-mapred</artifactId>
       <classifier>${avro.mapred.classifier}</classifier>
     </dependency>
+    <dependency>
+      <groupId>commons-httpclient</groupId>
+      <artifactId>commons-httpclient</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-avatica</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+    </dependency>
+    <!-- transitive dependencies of hive-exec-core doesn't declare -->
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.jodd</groupId>
+      <artifactId>jodd-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.datanucleus</groupId>
+      <artifactId>datanucleus-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libthrift</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libfb303</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.scalacheck</groupId>
       <artifactId>scalacheck_${scala.binary.version}</artifactId>
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 110f51a305861995a32a736270c7e7ecdd91523a..567d7fa12ff14b9b8e7bd67f95de4dd2530ff063 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
@@ -20,15 +20,18 @@ package org.apache.spark.sql.hive
 import java.io.File
 import java.net.{URL, URLClassLoader}
 import java.sql.Timestamp
+import java.util.concurrent.TimeUnit
 
 import scala.collection.JavaConversions._
 import scala.collection.mutable.HashMap
 import scala.language.implicitConversions
+import scala.concurrent.duration._
 
 import org.apache.hadoop.fs.{FileSystem, 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.conf.HiveConf.ConfVars
 import org.apache.hadoop.hive.ql.metadata.Table
 import org.apache.hadoop.hive.ql.parse.VariableSubstitution
 import org.apache.hadoop.hive.ql.session.SessionState
@@ -164,6 +167,16 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging {
   }
   SessionState.setCurrentSessionState(executionHive.state)
 
+  /**
+   * Overrides default Hive configurations to avoid breaking changes to Spark SQL users.
+   *  - allow SQL11 keywords to be used as identifiers
+   */
+  private[sql] def defaultOverides() = {
+    setConf(ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS.varname, "false")
+  }
+
+  defaultOverides()
+
   /**
    * The copy of the Hive client that is used to retrieve metadata from the Hive MetaStore.
    * The version of the Hive client that is used here must match the metastore that is configured
@@ -252,6 +265,10 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging {
   }
 
   protected[sql] override def parseSql(sql: String): LogicalPlan = {
+    var state = SessionState.get()
+    if (state == null) {
+      SessionState.setCurrentSessionState(tlSession.get().asInstanceOf[SQLSession].sessionState)
+    }
     super.parseSql(substitutor.substitute(hiveconf, sql))
   }
 
@@ -298,10 +315,21 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging {
         // Can we use fs.getContentSummary in future?
         // Seems fs.getContentSummary returns wrong table size on Jenkins. So we use
         // countFileSize to count the table size.
+        val stagingDir = metadataHive.getConf(HiveConf.ConfVars.STAGINGDIR.varname,
+          HiveConf.ConfVars.STAGINGDIR.defaultStrVal)
+
         def calculateTableSize(fs: FileSystem, path: Path): Long = {
           val fileStatus = fs.getFileStatus(path)
           val size = if (fileStatus.isDir) {
-            fs.listStatus(path).map(status => calculateTableSize(fs, status.getPath)).sum
+            fs.listStatus(path)
+              .map { status =>
+                if (!status.getPath().getName().startsWith(stagingDir)) {
+                  calculateTableSize(fs, status.getPath)
+                } else {
+                  0L
+                }
+              }
+              .sum
           } else {
             fileStatus.getLen
           }
@@ -398,7 +426,58 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging {
   }
 
   /** Overridden by child classes that need to set configuration before the client init. */
-  protected def configure(): Map[String, String] = Map.empty
+  protected def configure(): Map[String, String] = {
+    // Hive 0.14.0 introduces timeout operations in HiveConf, and changes default values of a bunch
+    // of time `ConfVar`s by adding time suffixes (`s`, `ms`, and `d` etc.).  This breaks backwards-
+    // compatibility when users are trying to connecting to a Hive metastore of lower version,
+    // because these options are expected to be integral values in lower versions of Hive.
+    //
+    // Here we enumerate all time `ConfVar`s and convert their values to numeric strings according
+    // to their output time units.
+    Seq(
+      ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY -> TimeUnit.SECONDS,
+      ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.METASTORE_CLIENT_SOCKET_LIFETIME -> TimeUnit.SECONDS,
+      ConfVars.HMSHANDLERINTERVAL -> TimeUnit.MILLISECONDS,
+      ConfVars.METASTORE_EVENT_DB_LISTENER_TTL -> TimeUnit.SECONDS,
+      ConfVars.METASTORE_EVENT_CLEAN_FREQ -> TimeUnit.SECONDS,
+      ConfVars.METASTORE_EVENT_EXPIRY_DURATION -> TimeUnit.SECONDS,
+      ConfVars.METASTORE_AGGREGATE_STATS_CACHE_TTL -> TimeUnit.SECONDS,
+      ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT -> TimeUnit.MILLISECONDS,
+      ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_STATS_JDBC_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.HIVE_STATS_RETRIES_WAIT -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES -> TimeUnit.SECONDS,
+      ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_TXN_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.HIVE_COMPACTOR_WORKER_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.HIVE_COMPACTOR_CHECK_INTERVAL -> TimeUnit.SECONDS,
+      ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS,
+      ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE -> TimeUnit.SECONDS,
+      ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS,
+      ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME -> TimeUnit.SECONDS,
+      ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS,
+      ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT -> TimeUnit.MILLISECONDS,
+      ConfVars.SERVER_READ_SOCKET_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL -> TimeUnit.MILLISECONDS,
+      ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.SPARK_JOB_MONITOR_TIMEOUT -> TimeUnit.SECONDS,
+      ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT -> TimeUnit.MILLISECONDS,
+      ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT -> TimeUnit.MILLISECONDS
+    ).map { case (confVar, unit) =>
+      confVar.varname -> hiveconf.getTimeVar(confVar, unit).toString
+    }.toMap
+  }
 
   protected[hive] class SQLSession extends super.SQLSession {
     protected[sql] override lazy val conf: SQLConf = new SQLConf {
@@ -515,19 +594,23 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging {
 
 private[hive] object HiveContext {
   /** The version of hive used internally by Spark SQL. */
-  val hiveExecutionVersion: String = "0.13.1"
+  val hiveExecutionVersion: String = "1.2.1"
 
   val HIVE_METASTORE_VERSION: String = "spark.sql.hive.metastore.version"
   val HIVE_METASTORE_JARS = stringConf("spark.sql.hive.metastore.jars",
     defaultValue = Some("builtin"),
-    doc = "Location of the jars that should be used to instantiate the HiveMetastoreClient. This" +
-      " property can be one of three options: " +
-      "1. \"builtin\" Use Hive 0.13.1, which is bundled with the Spark assembly jar when " +
-      "<code>-Phive</code> is enabled. When this option is chosen, " +
-      "spark.sql.hive.metastore.version must be either <code>0.13.1</code> or not defined. " +
-      "2. \"maven\" Use Hive jars of specified version downloaded from Maven repositories." +
-      "3. A classpath in the standard format for both Hive and Hadoop.")
-
+    doc = s"""
+      | Location of the jars that should be used to instantiate the HiveMetastoreClient.
+      | This property can be one of three options: "
+      | 1. "builtin"
+      |   Use Hive ${hiveExecutionVersion}, which is bundled with the Spark assembly jar when
+      |   <code>-Phive</code> is enabled. When this option is chosen,
+      |   <code>spark.sql.hive.metastore.version</code> must be either
+      |   <code>${hiveExecutionVersion}</code> or not defined.
+      | 2. "maven"
+      |   Use Hive jars of specified version downloaded from Maven repositories.
+      | 3. A classpath in the standard format for both Hive and Hadoop.
+    """.stripMargin)
   val CONVERT_METASTORE_PARQUET = booleanConf("spark.sql.hive.convertMetastoreParquet",
     defaultValue = Some(true),
     doc = "When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of " +
@@ -566,17 +649,18 @@ private[hive] object HiveContext {
   /** Constructs a configuration for hive, where the metastore is located in a temp directory. */
   def newTemporaryConfiguration(): Map[String, String] = {
     val tempDir = Utils.createTempDir()
-    val localMetastore = new File(tempDir, "metastore").getAbsolutePath
+    val localMetastore = new File(tempDir, "metastore")
     val propMap: HashMap[String, String] = HashMap()
     // We have to mask all properties in hive-site.xml that relates to metastore data source
     // as we used a local metastore here.
     HiveConf.ConfVars.values().foreach { confvar =>
       if (confvar.varname.contains("datanucleus") || confvar.varname.contains("jdo")) {
-        propMap.put(confvar.varname, confvar.defaultVal)
+        propMap.put(confvar.varname, confvar.getDefaultExpr())
       }
     }
-    propMap.put("javax.jdo.option.ConnectionURL",
-      s"jdbc:derby:;databaseName=$localMetastore;create=true")
+    propMap.put(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, localMetastore.toURI.toString)
+    propMap.put(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname,
+      s"jdbc:derby:;databaseName=${localMetastore.getAbsolutePath};create=true")
     propMap.put("datanucleus.rdbms.datastoreAdapterClassName",
       "org.datanucleus.store.rdbms.adapter.DerbyAdapter")
     propMap.toMap
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 a8c9b4fa71b9914b4554908bb9e1778eb928a248..16c186627f6cc12aee8b8714f749bc30dc69e363 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
@@ -649,11 +649,12 @@ private[hive] case class MetastoreRelation
     table.outputFormat.foreach(sd.setOutputFormat)
 
     val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo
-    sd.setSerdeInfo(serdeInfo)
     table.serde.foreach(serdeInfo.setSerializationLib)
+    sd.setSerdeInfo(serdeInfo)
+
     val serdeParameters = new java.util.HashMap[String, String]()
-    serdeInfo.setParameters(serdeParameters)
     table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) }
+    serdeInfo.setParameters(serdeParameters)
 
     new Table(tTable)
   }
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 e6df64d2642bced8c222727931b1d5204089ff71..e2fdfc6163a001420373a0288bc16b9587415ce9 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,6 +18,7 @@
 package org.apache.spark.sql.hive
 
 import java.sql.Date
+import java.util.Locale
 
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.serde.serdeConstants
@@ -80,6 +81,7 @@ private[hive] object HiveQl extends Logging {
     "TOK_ALTERDATABASE_PROPERTIES",
     "TOK_ALTERINDEX_PROPERTIES",
     "TOK_ALTERINDEX_REBUILD",
+    "TOK_ALTERTABLE",
     "TOK_ALTERTABLE_ADDCOLS",
     "TOK_ALTERTABLE_ADDPARTS",
     "TOK_ALTERTABLE_ALTERPARTS",
@@ -94,6 +96,7 @@ private[hive] object HiveQl extends Logging {
     "TOK_ALTERTABLE_SKEWED",
     "TOK_ALTERTABLE_TOUCH",
     "TOK_ALTERTABLE_UNARCHIVE",
+    "TOK_ALTERVIEW",
     "TOK_ALTERVIEW_ADDPARTS",
     "TOK_ALTERVIEW_AS",
     "TOK_ALTERVIEW_DROPPARTS",
@@ -248,7 +251,7 @@ private[hive] object HiveQl extends Logging {
      * Otherwise, there will be Null pointer exception,
      * when retrieving properties form HiveConf.
      */
-    val hContext = new Context(hiveConf)
+    val hContext = new Context(SessionState.get().getConf())
     val node = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql, hContext))
     hContext.clear()
     node
@@ -577,12 +580,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
             "TOK_TABLESKEWED", // Skewed by
             "TOK_TABLEROWFORMAT",
             "TOK_TABLESERIALIZER",
-            "TOK_FILEFORMAT_GENERIC", // For file formats not natively supported by Hive.
-            "TOK_TBLSEQUENCEFILE", // Stored as SequenceFile
-            "TOK_TBLTEXTFILE", // Stored as TextFile
-            "TOK_TBLRCFILE", // Stored as RCFile
-            "TOK_TBLORCFILE", // Stored as ORC File
-            "TOK_TBLPARQUETFILE", // Stored as PARQUET
+            "TOK_FILEFORMAT_GENERIC",
             "TOK_TABLEFILEFORMAT", // User-provided InputFormat and OutputFormat
             "TOK_STORAGEHANDLER", // Storage handler
             "TOK_TABLELOCATION",
@@ -706,36 +704,51 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
             tableDesc = tableDesc.copy(serdeProperties = tableDesc.serdeProperties ++ serdeParams)
           }
         case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) =>
-          throw new SemanticException(
-            "Unrecognized file format in STORED AS clause:${child.getText}")
+          child.getText().toLowerCase(Locale.ENGLISH) match {
+            case "orc" =>
+              tableDesc = tableDesc.copy(
+                inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"),
+                outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"))
+              if (tableDesc.serde.isEmpty) {
+                tableDesc = tableDesc.copy(
+                  serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde"))
+              }
 
-        case Token("TOK_TBLRCFILE", Nil) =>
-          tableDesc = tableDesc.copy(
-            inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"),
-            outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))
-          if (tableDesc.serde.isEmpty) {
-            tableDesc = tableDesc.copy(
-              serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"))
-          }
+            case "parquet" =>
+              tableDesc = tableDesc.copy(
+                inputFormat =
+                  Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
+                outputFormat =
+                  Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))
+              if (tableDesc.serde.isEmpty) {
+                tableDesc = tableDesc.copy(
+                  serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))
+              }
 
-        case Token("TOK_TBLORCFILE", Nil) =>
-          tableDesc = tableDesc.copy(
-            inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"),
-            outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"))
-          if (tableDesc.serde.isEmpty) {
-            tableDesc = tableDesc.copy(
-              serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde"))
-          }
+            case "rcfile" =>
+              tableDesc = tableDesc.copy(
+                inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"),
+                outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))
+              if (tableDesc.serde.isEmpty) {
+                tableDesc = tableDesc.copy(
+                  serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"))
+              }
 
-        case Token("TOK_TBLPARQUETFILE", Nil) =>
-          tableDesc = tableDesc.copy(
-            inputFormat =
-              Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
-            outputFormat =
-              Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))
-          if (tableDesc.serde.isEmpty) {
-            tableDesc = tableDesc.copy(
-              serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))
+            case "textfile" =>
+              tableDesc = tableDesc.copy(
+                inputFormat =
+                  Option("org.apache.hadoop.mapred.TextInputFormat"),
+                outputFormat =
+                  Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat"))
+
+            case "sequencefile" =>
+              tableDesc = tableDesc.copy(
+                inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"),
+                outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat"))
+
+            case _ =>
+              throw new SemanticException(
+                s"Unrecognized file format in STORED AS clause: ${child.getText}")
           }
 
         case Token("TOK_TABLESERIALIZER",
@@ -751,7 +764,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
 
         case Token("TOK_TABLEPROPERTIES", list :: Nil) =>
           tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list))
-        case list @ Token("TOK_TABLEFILEFORMAT", _) =>
+        case list @ Token("TOK_TABLEFILEFORMAT", children) =>
           tableDesc = tableDesc.copy(
             inputFormat =
               Option(BaseSemanticAnalyzer.unescapeSQLString(list.getChild(0).getText)),
@@ -889,7 +902,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
                 Token("TOK_TABLEPROPLIST", propsClause) :: Nil) :: Nil) :: Nil =>
                 val serdeProps = propsClause.map {
                   case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) =>
-                    (name, value)
+                    (BaseSemanticAnalyzer.unescapeSQLString(name),
+                      BaseSemanticAnalyzer.unescapeSQLString(value))
                 }
                 (Nil, Some(BaseSemanticAnalyzer.unescapeSQLString(serdeClass)), serdeProps)
 
@@ -1037,10 +1051,11 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
       // return With plan if there is CTE
       cteRelations.map(With(query, _)).getOrElse(query)
 
-    case Token("TOK_UNION", left :: right :: Nil) => Union(nodeToPlan(left), nodeToPlan(right))
+    // HIVE-9039 renamed TOK_UNION => TOK_UNIONALL while adding TOK_UNIONDISTINCT
+    case Token("TOK_UNIONALL", left :: right :: Nil) => Union(nodeToPlan(left), nodeToPlan(right))
 
     case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ")
+      throw new NotImplementedError(s"No parse rules for $node:\n ${dumpTree(a).toString} ")
   }
 
   val allJoinTokens = "(TOK_.*JOIN)".r
@@ -1251,7 +1266,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
       InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, true)
 
     case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ")
+      throw new NotImplementedError(s"No parse rules for ${a.getName}:" +
+          s"\n ${dumpTree(a).toString} ")
   }
 
   protected def selExprNodeToExpr(node: Node): Option[Expression] = node match {
@@ -1274,7 +1290,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
     case Token("TOK_HINTLIST", _) => None
 
     case a: ASTNode =>
-      throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ")
+      throw new NotImplementedError(s"No parse rules for ${a.getName }:" +
+          s"\n ${dumpTree(a).toString } ")
   }
 
   protected val escapedIdentifier = "`([^`]+)`".r
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
index a357bb39ca7fdf0623c6c8a7081200067e0dcdd9..267074f3ad1022a822cd8246e0d42066732596a8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
@@ -20,6 +20,8 @@ package org.apache.spark.sql.hive
 import java.io.{InputStream, OutputStream}
 import java.rmi.server.UID
 
+import org.apache.avro.Schema
+
 /* Implicit conversions */
 import scala.collection.JavaConversions._
 import scala.language.implicitConversions
@@ -33,7 +35,7 @@ import org.apache.hadoop.fs.Path
 import org.apache.hadoop.hive.ql.exec.{UDF, Utilities}
 import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils
-import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable
+import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils}
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector
 import org.apache.hadoop.io.Writable
 
@@ -82,10 +84,19 @@ private[hive] object HiveShim {
    * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that
    * is needed to initialize before serialization.
    */
-  def prepareWritable(w: Writable): Writable = {
+  def prepareWritable(w: Writable, serDeProps: Seq[(String, String)]): Writable = {
     w match {
       case w: AvroGenericRecordWritable =>
         w.setRecordReaderID(new UID())
+        // In Hive 1.1, the record's schema may need to be initialized manually or a NPE will
+        // be thrown.
+        if (w.getFileSchema() == null) {
+          serDeProps
+            .find(_._1 == AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName())
+            .foreach { kv =>
+              w.setFileSchema(new Schema.Parser().parse(kv._2))
+            }
+        }
       case _ =>
     }
     w
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala
index d834b4e83e0436c2189b9d479bde839a45601823..a82e152dcda2cb796222752d2232b4735afbf128 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala
@@ -87,6 +87,10 @@ private[hive] case class HiveTable(
  * shared classes.
  */
 private[hive] trait ClientInterface {
+
+  /** Returns the configuration for the given key in the current session. */
+  def getConf(key: String, defaultValue: String): String
+
   /**
    * Runs a HiveQL command using Hive, returning the results as a list of strings.  Each row will
    * result in one string.
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index 6e0912da5862da14b2f48f8ded0cf6fc651da009..dc372be0e5a3761785c2568fbc3bbd1280e0cd6d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -38,7 +38,6 @@ import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.execution.QueryExecutionException
 import org.apache.spark.util.{CircularBuffer, Utils}
 
-
 /**
  * A class that wraps the HiveClient and converts its responses to externally visible classes.
  * Note that this class is typically loaded with an internal classloader for each instantiation,
@@ -115,6 +114,10 @@ private[hive] class ClientWrapper(
   /** Returns the configuration for the current session. */
   def conf: HiveConf = SessionState.get().getConf
 
+  override def getConf(key: String, defaultValue: String): String = {
+    conf.get(key, defaultValue)
+  }
+
   // TODO: should be a def?s
   // When we create this val client, the HiveConf of it (conf) is the one associated with state.
   @GuardedBy("this")
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
index 956997e5f9dce45593eee5e14c64a041ae072dee..6e826ce552204c124590f682c4ddb78da3855e23 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
@@ -512,7 +512,7 @@ private[client] class Shim_v1_2 extends Shim_v1_1 {
       listBucketingEnabled: Boolean): Unit = {
     loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean,
       numDP: JInteger, holdDDLTime: JBoolean, listBucketingEnabled: JBoolean, JBoolean.FALSE,
-      0: JLong)
+      0L: JLong)
   }
 
 }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
index 97fb98199991bbea75b6a1590509e24d2bb6aca7..f58bc7d7a0af450bdc0a137aae0afddaf6d6afb8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
@@ -55,7 +55,7 @@ private[hive] object IsolatedClientLoader {
     case "14" | "0.14" | "0.14.0" => hive.v14
     case "1.0" | "1.0.0" => hive.v1_0
     case "1.1" | "1.1.0" => hive.v1_1
-    case "1.2" | "1.2.0" => hive.v1_2
+    case "1.2" | "1.2.0" | "1.2.1" => hive.v1_2
   }
 
   private def downloadVersion(version: HiveVersion, ivyPath: Option[String]): Seq[URL] = {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
index b48082fe4b36326192751d28065125f561d12060..0503691a44249f591b207fb57f9636b1dfcba7fa 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
@@ -56,7 +56,7 @@ package object client {
         "net.hydromatic:linq4j",
         "net.hydromatic:quidem"))
 
-    case object v1_2 extends HiveVersion("1.2.0",
+    case object v1_2 extends HiveVersion("1.2.1",
       exclusions = Seq("eigenbase:eigenbase-properties",
         "org.apache.curator:*",
         "org.pentaho:pentaho-aggdesigner-algorithm",
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 40a6a32156687008c7aaed117825495f23681933..12c667e6e92da7f3a616cec50cf4cb858fe7a541 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
@@ -129,7 +129,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.getExternalTmpPath(tableLocation.toUri)
+    val tmpLocation = hiveContext.getExternalTmpPath(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/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
index 7e3342cc84c0ee46268715176de7c5231fce7b68..fbb86406f40cbf54df3f074e68dc1563db95eeb7 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
@@ -247,7 +247,7 @@ private class ScriptTransformationWriterThread(
         } else {
           val writable = inputSerde.serialize(
             row.asInstanceOf[GenericInternalRow].values, inputSoi)
-          prepareWritable(writable).write(dataOutputStream)
+          prepareWritable(writable, ioschema.outputSerdeProps).write(dataOutputStream)
         }
       }
       outputStream.close()
@@ -345,9 +345,7 @@ case class HiveScriptIOSchema (
 
     val columnTypesNames = columnTypes.map(_.toTypeInfo.getTypeName()).mkString(",")
 
-    var propsMap = serdeProps.map(kv => {
-      (kv._1.split("'")(1), kv._2.split("'")(1))
-    }).toMap + (serdeConstants.LIST_COLUMNS -> columns.mkString(","))
+    var propsMap = serdeProps.toMap + (serdeConstants.LIST_COLUMNS -> columns.mkString(","))
     propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypesNames)
 
     val properties = new Properties()
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 abe5c6900313004261f404f946e6c40583a693a0..8a86a87368f29f891a094b8e373a639e84ad4d2d 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
@@ -249,7 +249,7 @@ private[spark] object ResolveHiveWindowFunction extends Rule[LogicalPlan] {
           // Get the class of this function.
           // In Hive 0.12, there is no windowFunctionInfo.getFunctionClass. So, we use
           // windowFunctionInfo.getfInfo().getFunctionClass for both Hive 0.13 and Hive 0.13.1.
-          val functionClass = windowFunctionInfo.getfInfo().getFunctionClass
+          val functionClass = windowFunctionInfo.getFunctionClass()
           val newChildren =
             // Rank(), DENSE_RANK(), CUME_DIST(), and PERCENT_RANK() do not take explicit
             // input parameters and requires implicit parameters, which
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 8850e060d2a737993f9086367a67c878c97f9f59..684ea1d137b4932ba263aacaf93e004058c3809e 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
@@ -171,7 +171,7 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer(
   import SparkHiveDynamicPartitionWriterContainer._
 
   private val defaultPartName = jobConf.get(
-    ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultVal)
+    ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultStrVal)
 
   @transient private var writers: mutable.HashMap[String, FileSinkOperator.RecordWriter] = _
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
index ddd5d24717addbdefaa6c20a93998b03ec4111c2..86142e5d66f37cec97559ddd6c1cec477d1b7432 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql.hive.orc
 
 import org.apache.hadoop.hive.common.`type`.{HiveChar, HiveDecimal, HiveVarchar}
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument
+import org.apache.hadoop.hive.ql.io.sarg.{SearchArgumentFactory, SearchArgument}
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder
 import org.apache.hadoop.hive.serde2.io.DateWritable
 
@@ -33,13 +33,13 @@ import org.apache.spark.sql.sources._
 private[orc] object OrcFilters extends Logging {
   def createFilter(expr: Array[Filter]): Option[SearchArgument] = {
     expr.reduceOption(And).flatMap { conjunction =>
-      val builder = SearchArgument.FACTORY.newBuilder()
+      val builder = SearchArgumentFactory.newBuilder()
       buildSearchArgument(conjunction, builder).map(_.build())
     }
   }
 
   private def buildSearchArgument(expression: Filter, builder: Builder): Option[Builder] = {
-    def newBuilder = SearchArgument.FACTORY.newBuilder()
+    def newBuilder = SearchArgumentFactory.newBuilder()
 
     def isSearchableLiteral(value: Any): Boolean = value match {
       // These are types recognized by the `SearchArgumentImpl.BuilderImpl.boxLiteral()` method.
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 7bbdef90cd6b9fff9ecabd2642aa2439992f6835..8d0bf46e8fad7154429672cc13162b1ac1edb176 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -20,29 +20,25 @@ package org.apache.spark.sql.hive.test
 import java.io.File
 import java.util.{Set => JavaSet}
 
-import org.apache.hadoop.hive.conf.HiveConf
+import scala.collection.mutable
+import scala.language.implicitConversions
+
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars
 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.parse.VariableSubstitution
 import org.apache.hadoop.hive.ql.processors._
 import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
 import org.apache.hadoop.hive.serde2.avro.AvroSerDe
 
-import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.CacheTableCommand
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.hive.execution.HiveNativeCommand
-import org.apache.spark.sql.SQLConf
 import org.apache.spark.util.Utils
 import org.apache.spark.{SparkConf, SparkContext}
 
-import scala.collection.mutable
-import scala.language.implicitConversions
-
 /* Implicit conversions */
 import scala.collection.JavaConversions._
 
@@ -83,15 +79,25 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
 
   hiveconf.set("hive.plan.serialization.format", "javaXML")
 
-  lazy val warehousePath = Utils.createTempDir()
+  lazy val warehousePath = Utils.createTempDir(namePrefix = "warehouse-")
+
+  lazy val scratchDirPath = {
+    val dir = Utils.createTempDir(namePrefix = "scratch-")
+    dir.delete()
+    dir
+  }
 
   private lazy val temporaryConfig = newTemporaryConfiguration()
 
   /** Sets up the system initially or after a RESET command */
-  protected override def configure(): Map[String, String] =
-    temporaryConfig ++ Map(
-      ConfVars.METASTOREWAREHOUSE.varname -> warehousePath.toString,
-      ConfVars.METASTORE_INTEGER_JDO_PUSHDOWN.varname -> "true")
+  protected override def configure(): Map[String, String] = {
+    super.configure() ++ temporaryConfig ++ Map(
+      ConfVars.METASTOREWAREHOUSE.varname -> warehousePath.toURI.toString,
+      ConfVars.METASTORE_INTEGER_JDO_PUSHDOWN.varname -> "true",
+      ConfVars.SCRATCHDIR.varname -> scratchDirPath.toURI.toString,
+      ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY.varname -> "1"
+    )
+  }
 
   val testTempDir = Utils.createTempDir()
 
@@ -244,7 +250,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
     }),
     TestTable("src_thrift", () => {
       import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer
-      import org.apache.hadoop.hive.serde2.thrift.test.Complex
       import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat}
       import org.apache.thrift.protocol.TBinaryProtocol
 
@@ -253,7 +258,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
          |CREATE TABLE src_thrift(fake INT)
          |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}'
          |WITH SERDEPROPERTIES(
-         |  'serialization.class'='${classOf[Complex].getName}',
+         |  'serialization.class'='org.apache.spark.sql.hive.test.Complex',
          |  'serialization.format'='${classOf[TBinaryProtocol].getName}'
          |)
          |STORED AS
@@ -437,6 +442,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
         case (k, v) =>
           metadataHive.runSqlHive(s"SET $k=$v")
       }
+      defaultOverides()
 
       runSqlHive("USE default")
 
diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java
new file mode 100644
index 0000000000000000000000000000000000000000..e010112bb93275118ab393961f7790874371d318
--- /dev/null
+++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java
@@ -0,0 +1,1139 @@
+/*
+ * 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.test;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.hadoop.hive.serde2.thrift.test.IntString;
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.EncodingUtils;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+
+/**
+ * This is a fork of Hive 0.13's org/apache/hadoop/hive/serde2/thrift/test/Complex.java, which
+ * does not contain union fields that are not supported by Spark SQL.
+ */
+
+@SuppressWarnings({"ALL", "unchecked"})
+public class Complex implements org.apache.thrift.TBase<Complex, Complex._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Complex");
+
+  private static final org.apache.thrift.protocol.TField AINT_FIELD_DESC = new org.apache.thrift.protocol.TField("aint", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField A_STRING_FIELD_DESC = new org.apache.thrift.protocol.TField("aString", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField LINT_FIELD_DESC = new org.apache.thrift.protocol.TField("lint", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField L_STRING_FIELD_DESC = new org.apache.thrift.protocol.TField("lString", org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField LINT_STRING_FIELD_DESC = new org.apache.thrift.protocol.TField("lintString", org.apache.thrift.protocol.TType.LIST, (short)5);
+  private static final org.apache.thrift.protocol.TField M_STRING_STRING_FIELD_DESC = new org.apache.thrift.protocol.TField("mStringString", org.apache.thrift.protocol.TType.MAP, (short)6);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ComplexStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ComplexTupleSchemeFactory());
+  }
+
+  private int aint; // required
+  private String aString; // required
+  private List<Integer> lint; // required
+  private List<String> lString; // required
+  private List<IntString> lintString; // required
+  private Map<String,String> mStringString; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    AINT((short)1, "aint"),
+    A_STRING((short)2, "aString"),
+    LINT((short)3, "lint"),
+    L_STRING((short)4, "lString"),
+    LINT_STRING((short)5, "lintString"),
+    M_STRING_STRING((short)6, "mStringString");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // AINT
+          return AINT;
+        case 2: // A_STRING
+          return A_STRING;
+        case 3: // LINT
+          return LINT;
+        case 4: // L_STRING
+          return L_STRING;
+        case 5: // LINT_STRING
+          return LINT_STRING;
+        case 6: // M_STRING_STRING
+          return M_STRING_STRING;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __AINT_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.AINT, new org.apache.thrift.meta_data.FieldMetaData("aint", org.apache.thrift.TFieldRequirementType.DEFAULT,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.A_STRING, new org.apache.thrift.meta_data.FieldMetaData("aString", org.apache.thrift.TFieldRequirementType.DEFAULT,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.LINT, new org.apache.thrift.meta_data.FieldMetaData("lint", org.apache.thrift.TFieldRequirementType.DEFAULT,
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
+    tmpMap.put(_Fields.L_STRING, new org.apache.thrift.meta_data.FieldMetaData("lString", org.apache.thrift.TFieldRequirementType.DEFAULT,
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.LINT_STRING, new org.apache.thrift.meta_data.FieldMetaData("lintString", org.apache.thrift.TFieldRequirementType.DEFAULT,
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, IntString.class))));
+    tmpMap.put(_Fields.M_STRING_STRING, new org.apache.thrift.meta_data.FieldMetaData("mStringString", org.apache.thrift.TFieldRequirementType.DEFAULT,
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING),
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Complex.class, metaDataMap);
+  }
+
+  public Complex() {
+  }
+
+  public Complex(
+    int aint,
+    String aString,
+    List<Integer> lint,
+    List<String> lString,
+    List<IntString> lintString,
+    Map<String,String> mStringString)
+  {
+    this();
+    this.aint = aint;
+    setAintIsSet(true);
+    this.aString = aString;
+    this.lint = lint;
+    this.lString = lString;
+    this.lintString = lintString;
+    this.mStringString = mStringString;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Complex(Complex other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.aint = other.aint;
+    if (other.isSetAString()) {
+      this.aString = other.aString;
+    }
+    if (other.isSetLint()) {
+      List<Integer> __this__lint = new ArrayList<Integer>();
+      for (Integer other_element : other.lint) {
+        __this__lint.add(other_element);
+      }
+      this.lint = __this__lint;
+    }
+    if (other.isSetLString()) {
+      List<String> __this__lString = new ArrayList<String>();
+      for (String other_element : other.lString) {
+        __this__lString.add(other_element);
+      }
+      this.lString = __this__lString;
+    }
+    if (other.isSetLintString()) {
+      List<IntString> __this__lintString = new ArrayList<IntString>();
+      for (IntString other_element : other.lintString) {
+        __this__lintString.add(new IntString(other_element));
+      }
+      this.lintString = __this__lintString;
+    }
+    if (other.isSetMStringString()) {
+      Map<String,String> __this__mStringString = new HashMap<String,String>();
+      for (Map.Entry<String, String> other_element : other.mStringString.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        String other_element_value = other_element.getValue();
+
+        String __this__mStringString_copy_key = other_element_key;
+
+        String __this__mStringString_copy_value = other_element_value;
+
+        __this__mStringString.put(__this__mStringString_copy_key, __this__mStringString_copy_value);
+      }
+      this.mStringString = __this__mStringString;
+    }
+  }
+
+  public Complex deepCopy() {
+    return new Complex(this);
+  }
+
+  @Override
+  public void clear() {
+    setAintIsSet(false);
+    this.aint = 0;
+    this.aString = null;
+    this.lint = null;
+    this.lString = null;
+    this.lintString = null;
+    this.mStringString = null;
+  }
+
+  public int getAint() {
+    return this.aint;
+  }
+
+  public void setAint(int aint) {
+    this.aint = aint;
+    setAintIsSet(true);
+  }
+
+  public void unsetAint() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __AINT_ISSET_ID);
+  }
+
+  /** Returns true if field aint is set (has been assigned a value) and false otherwise */
+  public boolean isSetAint() {
+    return EncodingUtils.testBit(__isset_bitfield, __AINT_ISSET_ID);
+  }
+
+  public void setAintIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __AINT_ISSET_ID, value);
+  }
+
+  public String getAString() {
+    return this.aString;
+  }
+
+  public void setAString(String aString) {
+    this.aString = aString;
+  }
+
+  public void unsetAString() {
+    this.aString = null;
+  }
+
+  /** Returns true if field aString is set (has been assigned a value) and false otherwise */
+  public boolean isSetAString() {
+    return this.aString != null;
+  }
+
+  public void setAStringIsSet(boolean value) {
+    if (!value) {
+      this.aString = null;
+    }
+  }
+
+  public int getLintSize() {
+    return (this.lint == null) ? 0 : this.lint.size();
+  }
+
+  public java.util.Iterator<Integer> getLintIterator() {
+    return (this.lint == null) ? null : this.lint.iterator();
+  }
+
+  public void addToLint(int elem) {
+    if (this.lint == null) {
+      this.lint = new ArrayList<>();
+    }
+    this.lint.add(elem);
+  }
+
+  public List<Integer> getLint() {
+    return this.lint;
+  }
+
+  public void setLint(List<Integer> lint) {
+    this.lint = lint;
+  }
+
+  public void unsetLint() {
+    this.lint = null;
+  }
+
+  /** Returns true if field lint is set (has been assigned a value) and false otherwise */
+  public boolean isSetLint() {
+    return this.lint != null;
+  }
+
+  public void setLintIsSet(boolean value) {
+    if (!value) {
+      this.lint = null;
+    }
+  }
+
+  public int getLStringSize() {
+    return (this.lString == null) ? 0 : this.lString.size();
+  }
+
+  public java.util.Iterator<String> getLStringIterator() {
+    return (this.lString == null) ? null : this.lString.iterator();
+  }
+
+  public void addToLString(String elem) {
+    if (this.lString == null) {
+      this.lString = new ArrayList<String>();
+    }
+    this.lString.add(elem);
+  }
+
+  public List<String> getLString() {
+    return this.lString;
+  }
+
+  public void setLString(List<String> lString) {
+    this.lString = lString;
+  }
+
+  public void unsetLString() {
+    this.lString = null;
+  }
+
+  /** Returns true if field lString is set (has been assigned a value) and false otherwise */
+  public boolean isSetLString() {
+    return this.lString != null;
+  }
+
+  public void setLStringIsSet(boolean value) {
+    if (!value) {
+      this.lString = null;
+    }
+  }
+
+  public int getLintStringSize() {
+    return (this.lintString == null) ? 0 : this.lintString.size();
+  }
+
+  public java.util.Iterator<IntString> getLintStringIterator() {
+    return (this.lintString == null) ? null : this.lintString.iterator();
+  }
+
+  public void addToLintString(IntString elem) {
+    if (this.lintString == null) {
+      this.lintString = new ArrayList<>();
+    }
+    this.lintString.add(elem);
+  }
+
+  public List<IntString> getLintString() {
+    return this.lintString;
+  }
+
+  public void setLintString(List<IntString> lintString) {
+    this.lintString = lintString;
+  }
+
+  public void unsetLintString() {
+    this.lintString = null;
+  }
+
+  /** Returns true if field lintString is set (has been assigned a value) and false otherwise */
+  public boolean isSetLintString() {
+    return this.lintString != null;
+  }
+
+  public void setLintStringIsSet(boolean value) {
+    if (!value) {
+      this.lintString = null;
+    }
+  }
+
+  public int getMStringStringSize() {
+    return (this.mStringString == null) ? 0 : this.mStringString.size();
+  }
+
+  public void putToMStringString(String key, String val) {
+    if (this.mStringString == null) {
+      this.mStringString = new HashMap<String,String>();
+    }
+    this.mStringString.put(key, val);
+  }
+
+  public Map<String,String> getMStringString() {
+    return this.mStringString;
+  }
+
+  public void setMStringString(Map<String,String> mStringString) {
+    this.mStringString = mStringString;
+  }
+
+  public void unsetMStringString() {
+    this.mStringString = null;
+  }
+
+  /** Returns true if field mStringString is set (has been assigned a value) and false otherwise */
+  public boolean isSetMStringString() {
+    return this.mStringString != null;
+  }
+
+  public void setMStringStringIsSet(boolean value) {
+    if (!value) {
+      this.mStringString = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case AINT:
+      if (value == null) {
+        unsetAint();
+      } else {
+        setAint((Integer)value);
+      }
+      break;
+
+    case A_STRING:
+      if (value == null) {
+        unsetAString();
+      } else {
+        setAString((String)value);
+      }
+      break;
+
+    case LINT:
+      if (value == null) {
+        unsetLint();
+      } else {
+        setLint((List<Integer>)value);
+      }
+      break;
+
+    case L_STRING:
+      if (value == null) {
+        unsetLString();
+      } else {
+        setLString((List<String>)value);
+      }
+      break;
+
+    case LINT_STRING:
+      if (value == null) {
+        unsetLintString();
+      } else {
+        setLintString((List<IntString>)value);
+      }
+      break;
+
+    case M_STRING_STRING:
+      if (value == null) {
+        unsetMStringString();
+      } else {
+        setMStringString((Map<String,String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case AINT:
+      return Integer.valueOf(getAint());
+
+    case A_STRING:
+      return getAString();
+
+    case LINT:
+      return getLint();
+
+    case L_STRING:
+      return getLString();
+
+    case LINT_STRING:
+      return getLintString();
+
+    case M_STRING_STRING:
+      return getMStringString();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case AINT:
+      return isSetAint();
+    case A_STRING:
+      return isSetAString();
+    case LINT:
+      return isSetLint();
+    case L_STRING:
+      return isSetLString();
+    case LINT_STRING:
+      return isSetLintString();
+    case M_STRING_STRING:
+      return isSetMStringString();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Complex)
+      return this.equals((Complex)that);
+    return false;
+  }
+
+  public boolean equals(Complex that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_aint = true;
+    boolean that_present_aint = true;
+    if (this_present_aint || that_present_aint) {
+      if (!(this_present_aint && that_present_aint))
+        return false;
+      if (this.aint != that.aint)
+        return false;
+    }
+
+    boolean this_present_aString = true && this.isSetAString();
+    boolean that_present_aString = true && that.isSetAString();
+    if (this_present_aString || that_present_aString) {
+      if (!(this_present_aString && that_present_aString))
+        return false;
+      if (!this.aString.equals(that.aString))
+        return false;
+    }
+
+    boolean this_present_lint = true && this.isSetLint();
+    boolean that_present_lint = true && that.isSetLint();
+    if (this_present_lint || that_present_lint) {
+      if (!(this_present_lint && that_present_lint))
+        return false;
+      if (!this.lint.equals(that.lint))
+        return false;
+    }
+
+    boolean this_present_lString = true && this.isSetLString();
+    boolean that_present_lString = true && that.isSetLString();
+    if (this_present_lString || that_present_lString) {
+      if (!(this_present_lString && that_present_lString))
+        return false;
+      if (!this.lString.equals(that.lString))
+        return false;
+    }
+
+    boolean this_present_lintString = true && this.isSetLintString();
+    boolean that_present_lintString = true && that.isSetLintString();
+    if (this_present_lintString || that_present_lintString) {
+      if (!(this_present_lintString && that_present_lintString))
+        return false;
+      if (!this.lintString.equals(that.lintString))
+        return false;
+    }
+
+    boolean this_present_mStringString = true && this.isSetMStringString();
+    boolean that_present_mStringString = true && that.isSetMStringString();
+    if (this_present_mStringString || that_present_mStringString) {
+      if (!(this_present_mStringString && that_present_mStringString))
+        return false;
+      if (!this.mStringString.equals(that.mStringString))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_aint = true;
+    builder.append(present_aint);
+    if (present_aint)
+      builder.append(aint);
+
+    boolean present_aString = true && (isSetAString());
+    builder.append(present_aString);
+    if (present_aString)
+      builder.append(aString);
+
+    boolean present_lint = true && (isSetLint());
+    builder.append(present_lint);
+    if (present_lint)
+      builder.append(lint);
+
+    boolean present_lString = true && (isSetLString());
+    builder.append(present_lString);
+    if (present_lString)
+      builder.append(lString);
+
+    boolean present_lintString = true && (isSetLintString());
+    builder.append(present_lintString);
+    if (present_lintString)
+      builder.append(lintString);
+
+    boolean present_mStringString = true && (isSetMStringString());
+    builder.append(present_mStringString);
+    if (present_mStringString)
+      builder.append(mStringString);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(Complex other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    Complex typedOther = (Complex)other;
+
+    lastComparison = Boolean.valueOf(isSetAint()).compareTo(typedOther.isSetAint());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAint()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aint, typedOther.aint);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetAString()).compareTo(typedOther.isSetAString());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAString()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aString, typedOther.aString);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetLint()).compareTo(typedOther.isSetLint());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetLint()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lint, typedOther.lint);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetLString()).compareTo(typedOther.isSetLString());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetLString()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lString, typedOther.lString);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetLintString()).compareTo(typedOther.isSetLintString());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetLintString()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lintString, typedOther.lintString);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetMStringString()).compareTo(typedOther.isSetMStringString());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMStringString()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mStringString, typedOther.mStringString);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Complex(");
+    boolean first = true;
+
+    sb.append("aint:");
+    sb.append(this.aint);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("aString:");
+    if (this.aString == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.aString);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("lint:");
+    if (this.lint == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.lint);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("lString:");
+    if (this.lString == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.lString);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("lintString:");
+    if (this.lintString == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.lintString);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("mStringString:");
+    if (this.mStringString == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.mStringString);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ComplexStandardSchemeFactory implements SchemeFactory {
+    public ComplexStandardScheme getScheme() {
+      return new ComplexStandardScheme();
+    }
+  }
+
+  private static class ComplexStandardScheme extends StandardScheme<Complex> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, Complex struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // AINT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.aint = iprot.readI32();
+              struct.setAintIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // A_STRING
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.aString = iprot.readString();
+              struct.setAStringIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // LINT
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list0 = iprot.readListBegin();
+                struct.lint = new ArrayList<Integer>(_list0.size);
+                for (int _i1 = 0; _i1 < _list0.size; ++_i1)
+                {
+                  int _elem2; // required
+                  _elem2 = iprot.readI32();
+                  struct.lint.add(_elem2);
+                }
+                iprot.readListEnd();
+              }
+              struct.setLintIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // L_STRING
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list3 = iprot.readListBegin();
+                struct.lString = new ArrayList<String>(_list3.size);
+                for (int _i4 = 0; _i4 < _list3.size; ++_i4)
+                {
+                  String _elem5; // required
+                  _elem5 = iprot.readString();
+                  struct.lString.add(_elem5);
+                }
+                iprot.readListEnd();
+              }
+              struct.setLStringIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // LINT_STRING
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list6 = iprot.readListBegin();
+                struct.lintString = new ArrayList<IntString>(_list6.size);
+                for (int _i7 = 0; _i7 < _list6.size; ++_i7)
+                {
+                  IntString _elem8; // required
+                  _elem8 = new IntString();
+                  _elem8.read(iprot);
+                  struct.lintString.add(_elem8);
+                }
+                iprot.readListEnd();
+              }
+              struct.setLintStringIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // M_STRING_STRING
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map9 = iprot.readMapBegin();
+                struct.mStringString = new HashMap<String,String>(2*_map9.size);
+                for (int _i10 = 0; _i10 < _map9.size; ++_i10)
+                {
+                  String _key11; // required
+                  String _val12; // required
+                  _key11 = iprot.readString();
+                  _val12 = iprot.readString();
+                  struct.mStringString.put(_key11, _val12);
+                }
+                iprot.readMapEnd();
+              }
+              struct.setMStringStringIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, Complex struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(AINT_FIELD_DESC);
+      oprot.writeI32(struct.aint);
+      oprot.writeFieldEnd();
+      if (struct.aString != null) {
+        oprot.writeFieldBegin(A_STRING_FIELD_DESC);
+        oprot.writeString(struct.aString);
+        oprot.writeFieldEnd();
+      }
+      if (struct.lint != null) {
+        oprot.writeFieldBegin(LINT_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.lint.size()));
+          for (int _iter13 : struct.lint)
+          {
+            oprot.writeI32(_iter13);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.lString != null) {
+        oprot.writeFieldBegin(L_STRING_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.lString.size()));
+          for (String _iter14 : struct.lString)
+          {
+            oprot.writeString(_iter14);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.lintString != null) {
+        oprot.writeFieldBegin(LINT_STRING_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.lintString.size()));
+          for (IntString _iter15 : struct.lintString)
+          {
+            _iter15.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.mStringString != null) {
+        oprot.writeFieldBegin(M_STRING_STRING_FIELD_DESC);
+        {
+          oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.mStringString.size()));
+          for (Map.Entry<String, String> _iter16 : struct.mStringString.entrySet())
+          {
+            oprot.writeString(_iter16.getKey());
+            oprot.writeString(_iter16.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ComplexTupleSchemeFactory implements SchemeFactory {
+    public ComplexTupleScheme getScheme() {
+      return new ComplexTupleScheme();
+    }
+  }
+
+  private static class ComplexTupleScheme extends TupleScheme<Complex> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, Complex struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetAint()) {
+        optionals.set(0);
+      }
+      if (struct.isSetAString()) {
+        optionals.set(1);
+      }
+      if (struct.isSetLint()) {
+        optionals.set(2);
+      }
+      if (struct.isSetLString()) {
+        optionals.set(3);
+      }
+      if (struct.isSetLintString()) {
+        optionals.set(4);
+      }
+      if (struct.isSetMStringString()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
+      if (struct.isSetAint()) {
+        oprot.writeI32(struct.aint);
+      }
+      if (struct.isSetAString()) {
+        oprot.writeString(struct.aString);
+      }
+      if (struct.isSetLint()) {
+        {
+          oprot.writeI32(struct.lint.size());
+          for (int _iter17 : struct.lint)
+          {
+            oprot.writeI32(_iter17);
+          }
+        }
+      }
+      if (struct.isSetLString()) {
+        {
+          oprot.writeI32(struct.lString.size());
+          for (String _iter18 : struct.lString)
+          {
+            oprot.writeString(_iter18);
+          }
+        }
+      }
+      if (struct.isSetLintString()) {
+        {
+          oprot.writeI32(struct.lintString.size());
+          for (IntString _iter19 : struct.lintString)
+          {
+            _iter19.write(oprot);
+          }
+        }
+      }
+      if (struct.isSetMStringString()) {
+        {
+          oprot.writeI32(struct.mStringString.size());
+          for (Map.Entry<String, String> _iter20 : struct.mStringString.entrySet())
+          {
+            oprot.writeString(_iter20.getKey());
+            oprot.writeString(_iter20.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, Complex struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(6);
+      if (incoming.get(0)) {
+        struct.aint = iprot.readI32();
+        struct.setAintIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.aString = iprot.readString();
+        struct.setAStringIsSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TList _list21 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+          struct.lint = new ArrayList<Integer>(_list21.size);
+          for (int _i22 = 0; _i22 < _list21.size; ++_i22)
+          {
+            int _elem23; // required
+            _elem23 = iprot.readI32();
+            struct.lint.add(_elem23);
+          }
+        }
+        struct.setLintIsSet(true);
+      }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TList _list24 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.lString = new ArrayList<String>(_list24.size);
+          for (int _i25 = 0; _i25 < _list24.size; ++_i25)
+          {
+            String _elem26; // required
+            _elem26 = iprot.readString();
+            struct.lString.add(_elem26);
+          }
+        }
+        struct.setLStringIsSet(true);
+      }
+      if (incoming.get(4)) {
+        {
+          org.apache.thrift.protocol.TList _list27 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.lintString = new ArrayList<IntString>(_list27.size);
+          for (int _i28 = 0; _i28 < _list27.size; ++_i28)
+          {
+            IntString _elem29; // required
+            _elem29 = new IntString();
+            _elem29.read(iprot);
+            struct.lintString.add(_elem29);
+          }
+        }
+        struct.setLintStringIsSet(true);
+      }
+      if (incoming.get(5)) {
+        {
+          org.apache.thrift.protocol.TMap _map30 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.mStringString = new HashMap<String,String>(2*_map30.size);
+          for (int _i31 = 0; _i31 < _map30.size; ++_i31)
+          {
+            String _key32; // required
+            String _val33; // required
+            _key32 = iprot.readString();
+            _val33 = iprot.readString();
+            struct.mStringString.put(_key32, _val33);
+          }
+        }
+        struct.setMStringStringIsSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java
index 64d1ce92931eb6d800e8816c1d63b12fca346161..15c2c3deb0d83f7a4db0ff5b873b25b5f56e3ec6 100644
--- a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java
+++ b/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java
@@ -90,8 +90,10 @@ public class JavaMetastoreDataSourcesSuite {
   @After
   public void tearDown() throws IOException {
     // Clean up tables.
-    sqlContext.sql("DROP TABLE IF EXISTS javaSavedTable");
-    sqlContext.sql("DROP TABLE IF EXISTS externalTable");
+    if (sqlContext != null) {
+      sqlContext.sql("DROP TABLE IF EXISTS javaSavedTable");
+      sqlContext.sql("DROP TABLE IF EXISTS externalTable");
+    }
   }
 
   @Test
diff --git a/sql/hive/src/test/resources/golden/! operator-0-ee7f6a60a9792041b85b18cda56429bf b/sql/hive/src/test/resources/golden/! operator-0-ee7f6a60a9792041b85b18cda56429bf
new file mode 100644
index 0000000000000000000000000000000000000000..d00491fd7e5bb6fa28c517a0bb32b8b506539d4d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/! operator-0-ee7f6a60a9792041b85b18cda56429bf	
@@ -0,0 +1 @@
+1
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 d35bf9093ca9c4132a69544a0e8de58851741b22..2383bef940973906bedad0bd3c871d68233cca96 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
@@ -15,9 +15,9 @@ my_enum_structlist_map	map<string,array<struct<my_string:string,my_enum:string,o
 my_stringlist       	array<string>       	from deserializer   
 my_structlist       	array<struct<my_string:string,my_enum:string,optionals:struct<>>>	from deserializer   
 my_enumlist         	array<string>       	from deserializer   
-my_stringset        	struct<>            	from deserializer   
-my_enumset          	struct<>            	from deserializer   
-my_structset        	struct<>            	from deserializer   
+my_stringset        	array<string>       	from deserializer   
+my_enumset          	array<string>       	from deserializer   
+my_structset        	array<struct<my_string:string,my_enum:string,optionals:struct<>>>	from deserializer   
 optionals           	struct<>            	from deserializer   
 b                   	string              	                    
 	 	 
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-41d474f5e6d7c61c36f74b4bec4e9e44
similarity index 100%
rename from sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189
rename to sql/hive/src/test/resources/golden/parenthesis_star_by-5-41d474f5e6d7c61c36f74b4bec4e9e44
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 501bb6ab32f25e4aae2686be9fab25f71cd5d5d2..7bb2c0ab439847ab975a90d4085efd5796b6585c 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,4 +1,4 @@
-CREATE  TABLE `tmp_showcrt1`(
+CREATE TABLE `tmp_showcrt1`(
   `key` smallint, 
   `value` float)
 COMMENT 'temporary table'
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 90f8415a1c6bea40f148d9ff79997cc72400b63b..3cc1a57ee3a477749b8ceb6f136c64bda51d1ed7 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,4 +1,4 @@
-CREATE  TABLE `tmp_feng.tmp_showcrt`(
+CREATE TABLE `tmp_feng.tmp_showcrt`(
   `key` string, 
   `value` int)
 ROW FORMAT SERDE 
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 4ee22e52303163bb8431b0a165da2e72e3045b94..b51c71a71f91ce28e6580739f627a1abfe6f867d 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,4 +1,4 @@
-CREATE  TABLE `tmp_showcrt1`(
+CREATE TABLE `tmp_showcrt1`(
   `key` int, 
   `value` string, 
   `newvalue` bigint)
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 6fda2570b53f1e5820b4e823741bd9cb53bc861e..29189e1d860a48dbb6f6196a528bd4944367a04f 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,4 +1,4 @@
-CREATE  TABLE `tmp_showcrt1`(
+CREATE TABLE `tmp_showcrt1`(
   `key` int, 
   `value` string, 
   `newvalue` bigint)
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 3049cd6243ad8eab127c297ff685dca6c913711c..1b283db3e7744aa8a1e07dd98f890141e13d551b 100644
--- a/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043
+++ b/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043
@@ -17,6 +17,7 @@
 ^
 abs
 acos
+add_months
 and
 array
 array_contains
@@ -29,6 +30,7 @@ base64
 between
 bin
 case
+cbrt
 ceil
 ceiling
 coalesce
@@ -47,7 +49,11 @@ covar_samp
 create_union
 cume_dist
 current_database
+current_date
+current_timestamp
+current_user
 date_add
+date_format
 date_sub
 datediff
 day
@@ -65,6 +71,7 @@ ewah_bitmap_empty
 ewah_bitmap_or
 exp
 explode
+factorial
 field
 find_in_set
 first_value
@@ -73,6 +80,7 @@ format_number
 from_unixtime
 from_utc_timestamp
 get_json_object
+greatest
 hash
 hex
 histogram_numeric
@@ -81,6 +89,7 @@ if
 in
 in_file
 index
+initcap
 inline
 instr
 isnotnull
@@ -88,10 +97,13 @@ isnull
 java_method
 json_tuple
 lag
+last_day
 last_value
 lcase
 lead
+least
 length
+levenshtein
 like
 ln
 locate
@@ -109,11 +121,15 @@ max
 min
 minute
 month
+months_between
 named_struct
 negative
+next_day
 ngrams
 noop
+noopstreaming
 noopwithmap
+noopwithmapstreaming
 not
 ntile
 nvl
@@ -147,10 +163,14 @@ rpad
 rtrim
 second
 sentences
+shiftleft
+shiftright
+shiftrightunsigned
 sign
 sin
 size
 sort_array
+soundex
 space
 split
 sqrt
@@ -170,6 +190,7 @@ to_unix_timestamp
 to_utc_timestamp
 translate
 trim
+trunc
 ucase
 unbase64
 unhex
diff --git a/sql/hive/src/test/resources/golden/show_tblproperties-1-be4adb893c7f946ebd76a648ce3cc1ae b/sql/hive/src/test/resources/golden/show_tblproperties-1-be4adb893c7f946ebd76a648ce3cc1ae
index 0f6cc6f44f1f7a5544a8d4996e81fb8b5dfbc462..fdf701f9628007ae0a8a47d967a6332ae5c5beb5 100644
--- a/sql/hive/src/test/resources/golden/show_tblproperties-1-be4adb893c7f946ebd76a648ce3cc1ae
+++ b/sql/hive/src/test/resources/golden/show_tblproperties-1-be4adb893c7f946ebd76a648ce3cc1ae
@@ -1 +1 @@
-Table tmpfoo does not have property: bar	 
+Table default.tmpfoo does not have property: bar	 
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 3c91e138d7bd58b07ec943044c137d055398052b..d8ec084f0b2b0659277bd43095aec06ff8df16ee 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
@@ -1,5 +1,5 @@
 date_add(start_date, num_days) - Returns the date that is num_days after start_date.
 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'
+   > SELECT date_add('2009-07-30', 1) FROM src LIMIT 1;
+  '2009-07-31'
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 29d663f35c5861b8707690d80d9e896628c0b1b1..169c5000362556e1ee634603c51a8682c93e68d2 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
@@ -1,5 +1,5 @@
 date_sub(start_date, num_days) - Returns the date that is num_days before start_date.
 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'
+   > SELECT date_sub('2009-07-30', 1) FROM src LIMIT 1;
+  '2009-07-29'
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 7ccaee7ad3bd478bd2cf2567e570a19bbf185256..42197f7ad3e517e6f1bff26f53d64d0c0b3c9d46 100644
--- a/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5
+++ b/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5
@@ -1,5 +1,5 @@
 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;
+   > SELECT datediff('2009-07-30', '2009-07-31') FROM src LIMIT 1;
   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 d4017178b4e6b50862dc8db50c1cf573ed4d7f9e..09703d10eab7ac4f37ba5664d80b0121f01b5047 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
+day(param) - Returns the day of the month of date/timestamp, or day component of interval
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 6135aafa50860491f95b356b5b1b8378b64fe45d..7c0ec1dc3be59525136fb809a5cd7432be413a8a 100644
--- a/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4
+++ b/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4
@@ -1,6 +1,9 @@
-day(date) - Returns the date of the month of date
+day(param) - Returns the day of the month of date/timestamp, or day component of interval
 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;
+param can be one of:
+1. A string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'.
+2. A date value
+3. A timestamp value
+4. A day-time interval valueExample:
+   > SELECT day('2009-07-30') FROM src LIMIT 1;
   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 47a7018d9d5acd47b3aaf9a6f3ef2e260c8a03cf..c37eb0ec2e969330bb7524e382ab316e529a1d8a 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
+dayofmonth(param) - Returns the day of the month of date/timestamp, or day component of interval
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 d9490e20a3b6dbb95600cac6ffb2ebb36ab28150..9e931f649914b00e31b13d109bd0592f27f277ee 100644
--- a/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8
+++ b/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8
@@ -1,6 +1,9 @@
-dayofmonth(date) - Returns the date of the month of date
+dayofmonth(param) - Returns the day of the month of date/timestamp, or day component of interval
 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;
+param can be one of:
+1. A string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'.
+2. A date value
+3. A timestamp value
+4. A day-time interval valueExample:
+   > SELECT dayofmonth('2009-07-30') FROM src LIMIT 1;
   30
diff --git a/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c b/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c
index 2cf0d9d61882e808b3a0b22d0b6a498f53181806..ce583fe81ff6844c54351934d7f733d40a7076a2 100644
--- a/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c
+++ b/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c
@@ -1 +1 @@
-There is no documentation for function 'if'
+IF(expr1,expr2,expr3) - If expr1 is TRUE (expr1 <> 0 and expr1 <> NULL) then IF() returns expr2; otherwise it returns expr3. IF() returns a numeric or string value, depending on the context in which it is used.
diff --git a/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a b/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a
index 2cf0d9d61882e808b3a0b22d0b6a498f53181806..ce583fe81ff6844c54351934d7f733d40a7076a2 100644
--- a/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a
+++ b/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a
@@ -1 +1 @@
-There is no documentation for function 'if'
+IF(expr1,expr2,expr3) - If expr1 is TRUE (expr1 <> 0 and expr1 <> NULL) then IF() returns expr2; otherwise it returns expr3. IF() returns a numeric or string value, depending on the context in which it is used.
diff --git a/sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c b/sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c
index 2cf0d9d61882e808b3a0b22d0b6a498f53181806..ce583fe81ff6844c54351934d7f733d40a7076a2 100644
--- a/sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c
+++ b/sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c
@@ -1 +1 @@
-There is no documentation for function 'if'
+IF(expr1,expr2,expr3) - If expr1 is TRUE (expr1 <> 0 and expr1 <> NULL) then IF() returns expr2; otherwise it returns expr3. IF() returns a numeric or string value, depending on the context in which it is used.
diff --git a/sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a b/sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a
index 2cf0d9d61882e808b3a0b22d0b6a498f53181806..ce583fe81ff6844c54351934d7f733d40a7076a2 100644
--- a/sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a
+++ b/sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a
@@ -1 +1 @@
-There is no documentation for function 'if'
+IF(expr1,expr2,expr3) - If expr1 is TRUE (expr1 <> 0 and expr1 <> NULL) then IF() returns expr2; otherwise it returns expr3. IF() returns a numeric or string value, depending on the context in which it is used.
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 231e4f382566dc0154898fb5c4c0f2cdad073c6b..06650592f8d3c8169b4561762b434f71cf0c965f 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
+minute(param) - Returns the minute component of the string/timestamp/interval
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 ea842ea174ae48809da65678d9e86fccc823ed81..08ddc19b84d82d9da816b836215e7eaf7b27b8f0 100644
--- a/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48
+++ b/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48
@@ -1,6 +1,8 @@
-minute(date) - Returns the minute of date
-date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'.
-Example:
+minute(param) - Returns the minute component of the string/timestamp/interval
+param can be one of:
+1. A string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'.
+2. A timestamp value
+3. A day-time interval valueExample:
    > SELECT minute('2009-07-30 12:58:59') FROM src LIMIT 1;
   58
   > SELECT minute('12:58:59') FROM src LIMIT 1;
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 231e4f382566dc0154898fb5c4c0f2cdad073c6b..06650592f8d3c8169b4561762b434f71cf0c965f 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
+minute(param) - Returns the minute component of the string/timestamp/interval
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 ea842ea174ae48809da65678d9e86fccc823ed81..08ddc19b84d82d9da816b836215e7eaf7b27b8f0 100644
--- a/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48
+++ b/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48
@@ -1,6 +1,8 @@
-minute(date) - Returns the minute of date
-date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'.
-Example:
+minute(param) - Returns the minute component of the string/timestamp/interval
+param can be one of:
+1. A string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'.
+2. A timestamp value
+3. A day-time interval valueExample:
    > SELECT minute('2009-07-30 12:58:59') FROM src LIMIT 1;
   58
   > SELECT minute('12:58:59') FROM src LIMIT 1;
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 d54ebfbd6fb1abb1d2239800f503c6d478d0313e..a529b107ff21666fd0122397808662fb48671339 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
+Synonyms: stddev, stddev_pop
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 5f674788180e8d0b5dd87d6701bbd64d7a781730..ac3176a382547e782487e8027a2fa6bd9079eea5 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
+Synonyms: std, stddev_pop
diff --git a/sql/hive/src/test/resources/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d b/sql/hive/src/test/resources/golden/union3-0-99620f72f0282904846a596ca5b3e46c
similarity index 100%
rename from sql/hive/src/test/resources/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d
rename to sql/hive/src/test/resources/golden/union3-0-99620f72f0282904846a596ca5b3e46c
diff --git a/sql/hive/src/test/resources/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 b/sql/hive/src/test/resources/golden/union3-2-90ca96ea59fd45cf0af8c020ae77c908
similarity index 100%
rename from sql/hive/src/test/resources/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9
rename to sql/hive/src/test/resources/golden/union3-2-90ca96ea59fd45cf0af8c020ae77c908
diff --git a/sql/hive/src/test/resources/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 b/sql/hive/src/test/resources/golden/union3-3-72b149ccaef751bcfe55d5ca37cb5fd7
similarity index 100%
rename from sql/hive/src/test/resources/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97
rename to sql/hive/src/test/resources/golden/union3-3-72b149ccaef751bcfe55d5ca37cb5fd7
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parenthesis_star_by.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parenthesis_star_by.q
index 9e036c1a91d3b7b6c409dd265dbe2522223e0b50..e911fbf2d2c5c5a795f290e40f7859dc29070157 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parenthesis_star_by.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parenthesis_star_by.q
@@ -5,6 +5,6 @@ SELECT * FROM (SELECT key, value FROM src DISTRIBUTE BY key, value)t ORDER BY ke
 
 
 SELECT key, value FROM src CLUSTER BY (key, value);
-SELECT key, value FROM src ORDER BY (key ASC, value ASC);
+SELECT key, value FROM src ORDER BY key ASC, value ASC;
 SELECT key, value FROM src SORT BY (key, value);
 SELECT * FROM (SELECT key, value FROM src DISTRIBUTE BY (key, value))t ORDER BY key, value;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union3.q
index b26a2e2799f7a2613e8b380761433ca4058d3f81..a989800cbf85169204101403a76c83b888fa4040 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union3.q
@@ -1,42 +1,41 @@
+-- SORT_QUERY_RESULTS
 
 explain
 SELECT *
 FROM (
   SELECT 1 AS id
   FROM (SELECT * FROM src LIMIT 1) s1
-  CLUSTER BY id
   UNION ALL
   SELECT 2 AS id
   FROM (SELECT * FROM src LIMIT 1) s1
-  CLUSTER BY id
   UNION ALL
   SELECT 3 AS id
   FROM (SELECT * FROM src LIMIT 1) s2
   UNION ALL
   SELECT 4 AS id
   FROM (SELECT * FROM src LIMIT 1) s2
+  CLUSTER BY id
 ) a;
 
 
 
 CREATE TABLE union_out (id int);
 
-insert overwrite table union_out 
+insert overwrite table union_out
 SELECT *
 FROM (
   SELECT 1 AS id
   FROM (SELECT * FROM src LIMIT 1) s1
-  CLUSTER BY id
   UNION ALL
   SELECT 2 AS id
   FROM (SELECT * FROM src LIMIT 1) s1
-  CLUSTER BY id
   UNION ALL
   SELECT 3 AS id
   FROM (SELECT * FROM src LIMIT 1) s2
   UNION ALL
   SELECT 4 AS id
   FROM (SELECT * FROM src LIMIT 1) s2
+  CLUSTER BY id
 ) a;
 
-select * from union_out cluster by id;
+select * from union_out;
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..34b2edb44b033b541f68283ceef7743a8a37dfba
--- /dev/null
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import java.net.URL
+
+import org.apache.spark.SparkFunSuite
+
+/**
+ * Verify that some classes load and that others are not found on the classpath.
+ *
+ *
+ * This is used to detect classpath and shading conflict, especially between
+ * Spark's required Kryo version and that which can be found in some Hive versions.
+ */
+class ClasspathDependenciesSuite extends SparkFunSuite {
+  private val classloader = this.getClass.getClassLoader
+
+  private def assertLoads(classname: String): Unit = {
+    val resourceURL: URL = Option(findResource(classname)).getOrElse {
+      fail(s"Class $classname not found as ${resourceName(classname)}")
+    }
+
+    logInfo(s"Class $classname at $resourceURL")
+    classloader.loadClass(classname)
+  }
+
+  private def assertLoads(classes: String*): Unit = {
+    classes.foreach(assertLoads)
+  }
+
+  private def findResource(classname: String): URL = {
+    val resource = resourceName(classname)
+    classloader.getResource(resource)
+  }
+
+  private def resourceName(classname: String): String = {
+    classname.replace(".", "/") + ".class"
+  }
+
+  private def assertClassNotFound(classname: String): Unit = {
+    Option(findResource(classname)).foreach { resourceURL =>
+      fail(s"Class $classname found at $resourceURL")
+    }
+
+    intercept[ClassNotFoundException] {
+      classloader.loadClass(classname)
+    }
+  }
+
+  private def assertClassNotFound(classes: String*): Unit = {
+    classes.foreach(assertClassNotFound)
+  }
+
+  private val KRYO = "com.esotericsoftware.kryo.Kryo"
+
+  private val SPARK_HIVE = "org.apache.hive."
+  private val SPARK_SHADED = "org.spark-project.hive.shaded."
+
+  test("shaded Protobuf") {
+    assertLoads(SPARK_SHADED + "com.google.protobuf.ServiceException")
+  }
+
+  test("hive-common") {
+    assertLoads("org.apache.hadoop.hive.conf.HiveConf")
+  }
+
+  test("hive-exec") {
+    assertLoads("org.apache.hadoop.hive.ql.CommandNeedRetryException")
+  }
+
+  private val STD_INSTANTIATOR = "org.objenesis.strategy.StdInstantiatorStrategy"
+
+  test("unshaded kryo") {
+    assertLoads(KRYO, STD_INSTANTIATOR)
+  }
+
+  test("Forbidden Dependencies") {
+    assertClassNotFound(
+      SPARK_HIVE + KRYO,
+      SPARK_SHADED + KRYO,
+      "org.apache.hive." + KRYO,
+      "com.esotericsoftware.shaded." + STD_INSTANTIATOR,
+      SPARK_HIVE + "com.esotericsoftware.shaded." + STD_INSTANTIATOR,
+      "org.apache.hive.com.esotericsoftware.shaded." + STD_INSTANTIATOR
+    )
+  }
+
+  test("parquet-hadoop-bundle") {
+    assertLoads(
+      "parquet.hadoop.ParquetOutputFormat",
+      "parquet.hadoop.ParquetInputFormat"
+    )
+  }
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
index 72b35959a491b5c21a2a988cdae6e81812ca9996..b8d41065d3f025b90ee2fc7f27e3c9b60e321671 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
@@ -19,8 +19,11 @@ package org.apache.spark.sql.hive
 
 import java.io.File
 
+import scala.collection.mutable.ArrayBuffer
 import scala.sys.process.{ProcessLogger, Process}
 
+import org.scalatest.exceptions.TestFailedDueToTimeoutException
+
 import org.apache.spark._
 import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext}
 import org.apache.spark.util.{ResetSystemProperties, Utils}
@@ -84,23 +87,39 @@ class HiveSparkSubmitSuite
   // This is copied from org.apache.spark.deploy.SparkSubmitSuite
   private def runSparkSubmit(args: Seq[String]): Unit = {
     val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+    val history = ArrayBuffer.empty[String]
+    val commands = Seq("./bin/spark-submit") ++ args
+    val commandLine = commands.mkString("'", "' '", "'")
     val process = Process(
-      Seq("./bin/spark-submit") ++ args,
+      commands,
       new File(sparkHome),
       "SPARK_TESTING" -> "1",
       "SPARK_HOME" -> sparkHome
     ).run(ProcessLogger(
       // scalastyle:off println
-      (line: String) => { println(s"out> $line") },
-      (line: String) => { println(s"err> $line") }
+      (line: String) => { println(s"stdout> $line"); history += s"out> $line"},
+      (line: String) => { println(s"stderr> $line"); history += s"err> $line" }
       // scalastyle:on println
     ))
 
     try {
-      val exitCode = failAfter(180 seconds) { process.exitValue() }
+      val exitCode = failAfter(180.seconds) { process.exitValue() }
       if (exitCode != 0) {
-        fail(s"Process returned with exit code $exitCode. See the log4j logs for more detail.")
+        // include logs in output. Note that logging is async and may not have completed
+        // at the time this exception is raised
+        Thread.sleep(1000)
+        val historyLog = history.mkString("\n")
+        fail(s"$commandLine returned with exit code $exitCode." +
+            s" See the log4j logs for more detail." +
+            s"\n$historyLog")
       }
+    } catch {
+      case to: TestFailedDueToTimeoutException =>
+        val historyLog = history.mkString("\n")
+        fail(s"Timeout of $commandLine" +
+            s" See the log4j logs for more detail." +
+            s"\n$historyLog", to)
+        case t: Throwable => throw t
     } finally {
       // Ensure we still kill the process in case it timed out
       process.destroy()
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 508695919e9a70e0b6c6280b9fe6cda90ba129c6..d33e81227db885edd16436587f3cf449f5046bb1 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql.hive
 
 import java.io.File
 
+import org.apache.hadoop.hive.conf.HiveConf
 import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.sql.execution.QueryExecutionException
@@ -113,6 +114,8 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter {
   test("SPARK-4203:random partition directory order") {
     sql("CREATE TABLE tmp_table (key int, value string)")
     val tmpDir = Utils.createTempDir()
+    val stagingDir = new HiveConf().getVar(HiveConf.ConfVars.STAGINGDIR)
+
     sql(
       s"""
          |CREATE TABLE table_with_partition(c1 string)
@@ -145,7 +148,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter {
       """.stripMargin)
     def listFolders(path: File, acc: List[String]): List[List[String]] = {
       val dir = path.listFiles()
-      val folders = dir.filter(_.isDirectory).toList
+      val folders = dir.filter { e => e.isDirectory && !e.getName().startsWith(stagingDir) }.toList
       if (folders.isEmpty) {
         List(acc.reverse)
       } else {
@@ -158,7 +161,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter {
       "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=1"::Nil ,
       "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=4"::Nil
     )
-    assert(listFolders(tmpDir, List()).sortBy(_.toString()) == expected.sortBy(_.toString))
+    assert(listFolders(tmpDir, List()).sortBy(_.toString()) === expected.sortBy(_.toString))
     sql("DROP TABLE table_with_partition")
     sql("DROP TABLE tmp_table")
   }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
index bb5f1febe9ad4beab9fd5bbac9abc86ab8533624..f00d3754c364a824116d7c25515acf4b277e1619 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.hive
 
+import org.apache.hadoop.hive.conf.HiveConf
+
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.parquet.ParquetCompatibilityTest
 import org.apache.spark.sql.{Row, SQLConf, SQLContext}
@@ -26,6 +28,13 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest {
 
   override val sqlContext: SQLContext = TestHive
 
+  /**
+   * Set the staging directory (and hence path to ignore Parquet files under)
+   * to that set by [[HiveConf.ConfVars.STAGINGDIR]].
+   */
+  override val stagingDir: Option[String] =
+    Some(new HiveConf().getVar(HiveConf.ConfVars.STAGINGDIR))
+
   override protected def beforeAll(): Unit = {
     super.beforeAll()
 
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 bc72b0172a4673801852d9cfb92df1150d207c34..e4fec7e2c8a2a5dbfbbb0508fcc92cad5cd97d47 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
@@ -54,6 +54,9 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
       }
     }
 
+    // Ensure session state is initialized.
+    ctx.parseSql("use default")
+
     assertAnalyzeCommand(
       "ANALYZE TABLE Table1 COMPUTE STATISTICS",
       classOf[HiveNativeCommand])
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
index 3eb127e23d486c6c0ca6620050f804d8df104d80..f0bb77092c0cf1608c32a640430992de866774d0 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.client
 
 import java.io.File
 
+import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.{Logging, SparkFunSuite}
 import org.apache.spark.sql.catalyst.expressions.{NamedExpression, Literal, AttributeReference, EqualTo}
 import org.apache.spark.sql.catalyst.util.quietly
@@ -48,7 +49,9 @@ class VersionsSuite extends SparkFunSuite with Logging {
   }
 
   test("success sanity check") {
-    val badClient = IsolatedClientLoader.forVersion("13", buildConf(), ivyPath).client
+    val badClient = IsolatedClientLoader.forVersion(HiveContext.hiveExecutionVersion,
+      buildConf(),
+      ivyPath).client
     val db = new HiveDatabase("default", "")
     badClient.createDatabase(db)
   }
@@ -91,6 +94,7 @@ class VersionsSuite extends SparkFunSuite with Logging {
   versions.foreach { version =>
     test(s"$version: create client") {
       client = null
+      System.gc() // Hack to avoid SEGV on some JVM versions.
       client = IsolatedClientLoader.forVersion(version, buildConf(), ivyPath).client
     }
 
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 11a843becce69ce432b953c2f75f908d26786e62..a7cfac51cc0978701a9c5a861917540c04f2ef9e 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
@@ -52,14 +52,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
     TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles"))
     // Add Locale setting
     Locale.setDefault(Locale.US)
-    sql(s"ADD JAR ${TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath()}")
-    // The function source code can be found at:
-    // https://cwiki.apache.org/confluence/display/Hive/DeveloperGuide+UDTF
-    sql(
-      """
-        |CREATE TEMPORARY FUNCTION udtf_count2
-        |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
-      """.stripMargin)
   }
 
   override def afterAll() {
@@ -69,15 +61,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
     sql("DROP TEMPORARY FUNCTION udtf_count2")
   }
 
-  createQueryTest("Test UDTF.close in Lateral Views",
-     """
-       |SELECT key, cc
-       |FROM src LATERAL VIEW udtf_count2(value) dd AS cc
-     """.stripMargin, false) // false mean we have to keep the temp function in registry
-
-  createQueryTest("Test UDTF.close in SELECT",
-     "SELECT udtf_count2(a) FROM (SELECT 1 AS a FROM src LIMIT 3) table", false)
-
   test("SPARK-4908: concurrent hive native commands") {
     (1 to 100).par.map { _ =>
       sql("USE default")
@@ -176,8 +159,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
   createQueryTest("! operator",
     """
       |SELECT a FROM (
-      |  SELECT 1 AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT 2 AS a FROM src LIMIT 1) table
+      |  SELECT 1 AS a UNION ALL SELECT 2 AS a) t
       |WHERE !(a>1)
     """.stripMargin)
 
@@ -229,71 +211,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
     |FROM src LIMIT 1;
   """.stripMargin)
 
-  createQueryTest("count distinct 0 values",
-    """
-      |SELECT COUNT(DISTINCT a) FROM (
-      |  SELECT 'a' AS a FROM src LIMIT 0) table
-    """.stripMargin)
-
-  createQueryTest("count distinct 1 value strings",
-    """
-      |SELECT COUNT(DISTINCT a) FROM (
-      |  SELECT 'a' AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT 'b' AS a FROM src LIMIT 1) table
-    """.stripMargin)
-
-  createQueryTest("count distinct 1 value",
-    """
-      |SELECT COUNT(DISTINCT a) FROM (
-      |  SELECT 1 AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT 1 AS a FROM src LIMIT 1) table
-    """.stripMargin)
-
-  createQueryTest("count distinct 2 values",
-    """
-      |SELECT COUNT(DISTINCT a) FROM (
-      |  SELECT 1 AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT 2 AS a FROM src LIMIT 1) table
-    """.stripMargin)
-
-  createQueryTest("count distinct 2 values including null",
-    """
-      |SELECT COUNT(DISTINCT a, 1) FROM (
-      |  SELECT 1 AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT 1 AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT null AS a FROM src LIMIT 1) table
-    """.stripMargin)
-
-  createQueryTest("count distinct 1 value + null",
-  """
-    |SELECT COUNT(DISTINCT a) FROM (
-    |  SELECT 1 AS a FROM src LIMIT 1 UNION ALL
-    |  SELECT 1 AS a FROM src LIMIT 1 UNION ALL
-    |  SELECT null AS a FROM src LIMIT 1) table
-  """.stripMargin)
-
-  createQueryTest("count distinct 1 value long",
-    """
-      |SELECT COUNT(DISTINCT a) FROM (
-      |  SELECT 1L AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT 1L AS a FROM src LIMIT 1) table
-    """.stripMargin)
-
-  createQueryTest("count distinct 2 values long",
-    """
-      |SELECT COUNT(DISTINCT a) FROM (
-      |  SELECT 1L AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT 2L AS a FROM src LIMIT 1) table
-    """.stripMargin)
-
-  createQueryTest("count distinct 1 value + null long",
-    """
-      |SELECT COUNT(DISTINCT a) FROM (
-      |  SELECT 1L AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT 1L AS a FROM src LIMIT 1 UNION ALL
-      |  SELECT null AS a FROM src LIMIT 1) table
-    """.stripMargin)
-
   createQueryTest("null case",
     "SELECT case when(true) then 1 else null end FROM src LIMIT 1")
 
@@ -674,7 +591,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
     sql(
       """
         |SELECT a FROM (
-        |  SELECT 1 AS a FROM src LIMIT 1 ) table
+        |  SELECT 1 AS a FROM src LIMIT 1 ) t
         |WHERE abs(20141202) is not null
       """.stripMargin).collect()
   }
@@ -987,7 +904,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
         .zip(parts)
         .map { case (k, v) =>
           if (v == "NULL") {
-            s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultVal}"
+            s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultStrVal}"
           } else {
             s"$k=$v"
           }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
index e83a7dc77e329afc70fce9f93304119435848fa9..3bf8f3ac20480e5bb98a10d5610ceacd3bc32048 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
@@ -82,16 +82,16 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter {
     Seq.empty)
 
   createPruningTest("Column pruning - non-trivial top project with aliases",
-    "SELECT c1 * 2 AS double FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3",
-    Seq("double"),
+    "SELECT c1 * 2 AS dbl FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3",
+    Seq("dbl"),
     Seq("key"),
     Seq.empty)
 
   // Partition pruning tests
 
   createPruningTest("Partition pruning - non-partitioned, non-trivial project",
-    "SELECT key * 2 AS double FROM src WHERE value IS NOT NULL",
-    Seq("double"),
+    "SELECT key * 2 AS dbl FROM src WHERE value IS NOT NULL",
+    Seq("dbl"),
     Seq("key", "value"),
     Seq.empty)
 
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 c4923d83e48f35e84511843ea087bd1bb1eed983..95c1da6e9796caaa8d0e7172829704c8e1cf70f0 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
@@ -67,6 +67,25 @@ class MyDialect extends DefaultParserDialect
 class SQLQuerySuite extends QueryTest with SQLTestUtils {
   override def sqlContext: SQLContext = TestHive
 
+  test("UDTF") {
+    sql(s"ADD JAR ${TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath()}")
+    // The function source code can be found at:
+    // https://cwiki.apache.org/confluence/display/Hive/DeveloperGuide+UDTF
+    sql(
+      """
+        |CREATE TEMPORARY FUNCTION udtf_count2
+        |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
+      """.stripMargin)
+
+    checkAnswer(
+      sql("SELECT key, cc FROM src LATERAL VIEW udtf_count2(value) dd AS cc"),
+      Row(97, 500) :: Row(97, 500) :: Nil)
+
+    checkAnswer(
+      sql("SELECT udtf_count2(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"),
+      Row(3) :: Row(3) :: Nil)
+  }
+
   test("SPARK-6835: udtf in lateral view") {
     val df = Seq((1, 1)).toDF("c1", "c2")
     df.registerTempTable("table1")
@@ -264,47 +283,51 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils {
 
     setConf(HiveContext.CONVERT_CTAS, true)
 
-    sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
-    sql("CREATE TABLE IF NOT EXISTS ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
-    var message = intercept[AnalysisException] {
+    try {
       sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
-    }.getMessage
-    assert(message.contains("ctas1 already exists"))
-    checkRelation("ctas1", true)
-    sql("DROP TABLE ctas1")
-
-    // Specifying database name for query can be converted to data source write path
-    // is not allowed right now.
-    message = intercept[AnalysisException] {
-      sql("CREATE TABLE default.ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
-    }.getMessage
-    assert(
-      message.contains("Cannot specify database name in a CTAS statement"),
-      "When spark.sql.hive.convertCTAS is true, we should not allow " +
-      "database name specified.")
-
-    sql("CREATE TABLE ctas1 stored as textfile AS SELECT key k, value FROM src ORDER BY k, value")
-    checkRelation("ctas1", true)
-    sql("DROP TABLE ctas1")
-
-    sql(
-      "CREATE TABLE ctas1 stored as sequencefile AS SELECT key k, value FROM src ORDER BY k, value")
-    checkRelation("ctas1", true)
-    sql("DROP TABLE ctas1")
-
-    sql("CREATE TABLE ctas1 stored as rcfile AS SELECT key k, value FROM src ORDER BY k, value")
-    checkRelation("ctas1", false)
-    sql("DROP TABLE ctas1")
-
-    sql("CREATE TABLE ctas1 stored as orc AS SELECT key k, value FROM src ORDER BY k, value")
-    checkRelation("ctas1", false)
-    sql("DROP TABLE ctas1")
-
-    sql("CREATE TABLE ctas1 stored as parquet AS SELECT key k, value FROM src ORDER BY k, value")
-    checkRelation("ctas1", false)
-    sql("DROP TABLE ctas1")
-
-    setConf(HiveContext.CONVERT_CTAS, originalConf)
+      sql("CREATE TABLE IF NOT EXISTS ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
+      var message = intercept[AnalysisException] {
+        sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
+      }.getMessage
+      assert(message.contains("ctas1 already exists"))
+      checkRelation("ctas1", true)
+      sql("DROP TABLE ctas1")
+
+      // Specifying database name for query can be converted to data source write path
+      // is not allowed right now.
+      message = intercept[AnalysisException] {
+        sql("CREATE TABLE default.ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
+      }.getMessage
+      assert(
+        message.contains("Cannot specify database name in a CTAS statement"),
+        "When spark.sql.hive.convertCTAS is true, we should not allow " +
+            "database name specified.")
+
+      sql("CREATE TABLE ctas1 stored as textfile" +
+          " AS SELECT key k, value FROM src ORDER BY k, value")
+      checkRelation("ctas1", true)
+      sql("DROP TABLE ctas1")
+
+      sql("CREATE TABLE ctas1 stored as sequencefile" +
+            " AS SELECT key k, value FROM src ORDER BY k, value")
+      checkRelation("ctas1", true)
+      sql("DROP TABLE ctas1")
+
+      sql("CREATE TABLE ctas1 stored as rcfile AS SELECT key k, value FROM src ORDER BY k, value")
+      checkRelation("ctas1", false)
+      sql("DROP TABLE ctas1")
+
+      sql("CREATE TABLE ctas1 stored as orc AS SELECT key k, value FROM src ORDER BY k, value")
+      checkRelation("ctas1", false)
+      sql("DROP TABLE ctas1")
+
+      sql("CREATE TABLE ctas1 stored as parquet AS SELECT key k, value FROM src ORDER BY k, value")
+      checkRelation("ctas1", false)
+      sql("DROP TABLE ctas1")
+    } finally {
+      setConf(HiveContext.CONVERT_CTAS, originalConf)
+      sql("DROP TABLE IF EXISTS ctas1")
+    }
   }
 
   test("SQL Dialect Switching") {
@@ -670,22 +693,25 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils {
     val originalConf = convertCTAS
     setConf(HiveContext.CONVERT_CTAS, false)
 
-    sql("CREATE TABLE explodeTest (key bigInt)")
-    table("explodeTest").queryExecution.analyzed match {
-      case metastoreRelation: MetastoreRelation => // OK
-      case _ =>
-        fail("To correctly test the fix of SPARK-5875, explodeTest should be a MetastoreRelation")
-    }
+    try {
+      sql("CREATE TABLE explodeTest (key bigInt)")
+      table("explodeTest").queryExecution.analyzed match {
+        case metastoreRelation: MetastoreRelation => // OK
+        case _ =>
+          fail("To correctly test the fix of SPARK-5875, explodeTest should be a MetastoreRelation")
+      }
 
-    sql(s"INSERT OVERWRITE TABLE explodeTest SELECT explode(a) AS val FROM data")
-    checkAnswer(
-      sql("SELECT key from explodeTest"),
-      (1 to 5).flatMap(i => Row(i) :: Row(i + 1) :: Nil)
-    )
+      sql(s"INSERT OVERWRITE TABLE explodeTest SELECT explode(a) AS val FROM data")
+      checkAnswer(
+        sql("SELECT key from explodeTest"),
+        (1 to 5).flatMap(i => Row(i) :: Row(i + 1) :: Nil)
+      )
 
-    sql("DROP TABLE explodeTest")
-    dropTempTable("data")
-    setConf(HiveContext.CONVERT_CTAS, originalConf)
+      sql("DROP TABLE explodeTest")
+      dropTempTable("data")
+    } finally {
+      setConf(HiveContext.CONVERT_CTAS, originalConf)
+    }
   }
 
   test("sanity test for SPARK-6618") {
@@ -1058,12 +1084,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils {
   test("SPARK-8588 HiveTypeCoercion.inConversion fires too early") {
     val df =
       TestHive.createDataFrame(Seq((1, "2014-01-01"), (2, "2015-01-01"), (3, "2016-01-01")))
-    df.toDF("id", "date").registerTempTable("test_SPARK8588")
+    df.toDF("id", "datef").registerTempTable("test_SPARK8588")
     checkAnswer(
       TestHive.sql(
         """
-          |select id, concat(year(date))
-          |from test_SPARK8588 where concat(year(date), ' year') in ('2015 year', '2014 year')
+          |select id, concat(year(datef))
+          |from test_SPARK8588 where concat(year(datef), ' year') in ('2015 year', '2014 year')
         """.stripMargin),
       Row(1, "2014") :: Row(2, "2015") :: Nil
     )
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
index af3f468aaa5e9963ae092cde9e98901f19fb05db..deec0048d24b8d7e2bad826090061192017b40a8 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
@@ -48,11 +48,9 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest {
         StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true))
 
       checkQueries(
-        load(
-          source = dataSourceName,
-          options = Map(
-            "path" -> file.getCanonicalPath,
-            "dataSchema" -> dataSchemaWithPartition.json)))
+        read.options(Map(
+          "path" -> file.getCanonicalPath,
+          "dataSchema" -> dataSchemaWithPartition.json)).format(dataSourceName).load())
     }
   }
 }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
index d463e8fd626f975ab16b70c3ddc7cb01ce5d2f9a..a46ca9a2c9706d2ec2822953d2f13db364252c45 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
@@ -31,7 +31,6 @@ import org.scalatest.BeforeAndAfterAll
 import scala.reflect.ClassTag
 import scala.reflect.runtime.universe.TypeTag
 
-
 // The data where the partitioning key exists only in the directory structure.
 case class OrcParData(intField: Int, stringField: String)
 
@@ -40,7 +39,7 @@ case class OrcParDataWithKey(intField: Int, pi: Int, stringField: String, ps: St
 
 // TODO This test suite duplicates ParquetPartitionDiscoverySuite a lot
 class OrcPartitionDiscoverySuite extends QueryTest with BeforeAndAfterAll {
-  val defaultPartitionName = ConfVars.DEFAULTPARTITIONNAME.defaultVal
+  val defaultPartitionName = ConfVars.DEFAULTPARTITIONNAME.defaultStrVal
 
   def withTempDir(f: File => Unit): Unit = {
     val dir = Utils.createTempDir().getCanonicalFile
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
index f56fb96c52d37def3df5b019522155e129fbd7db..c4bc60086f6e197741b42f2a89cc351e08414551 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
@@ -60,7 +60,14 @@ case class ParquetDataWithKeyAndComplexTypes(
 class ParquetMetastoreSuite extends ParquetPartitioningTest {
   override def beforeAll(): Unit = {
     super.beforeAll()
-
+    dropTables("partitioned_parquet",
+      "partitioned_parquet_with_key",
+      "partitioned_parquet_with_complextypes",
+      "partitioned_parquet_with_key_and_complextypes",
+      "normal_parquet",
+      "jt",
+      "jt_array",
+      "test_parquet")
     sql(s"""
       create external table partitioned_parquet
       (
@@ -172,14 +179,14 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
   }
 
   override def afterAll(): Unit = {
-    sql("DROP TABLE partitioned_parquet")
-    sql("DROP TABLE partitioned_parquet_with_key")
-    sql("DROP TABLE partitioned_parquet_with_complextypes")
-    sql("DROP TABLE partitioned_parquet_with_key_and_complextypes")
-    sql("DROP TABLE normal_parquet")
-    sql("DROP TABLE IF EXISTS jt")
-    sql("DROP TABLE IF EXISTS jt_array")
-    sql("DROP TABLE IF EXISTS test_parquet")
+    dropTables("partitioned_parquet",
+      "partitioned_parquet_with_key",
+      "partitioned_parquet_with_complextypes",
+      "partitioned_parquet_with_key_and_complextypes",
+      "normal_parquet",
+      "jt",
+      "jt_array",
+       "test_parquet")
     setConf(HiveContext.CONVERT_METASTORE_PARQUET, false)
   }
 
@@ -203,6 +210,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
   }
 
   test("insert into an empty parquet table") {
+    dropTables("test_insert_parquet")
     sql(
       """
         |create table test_insert_parquet
@@ -228,7 +236,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
       sql(s"SELECT intField, stringField FROM test_insert_parquet WHERE intField > 2"),
       Row(3, "str3") :: Row(4, "str4") :: Nil
     )
-    sql("DROP TABLE IF EXISTS test_insert_parquet")
+    dropTables("test_insert_parquet")
 
     // Create it again.
     sql(
@@ -255,118 +263,118 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
       sql(s"SELECT intField, stringField FROM test_insert_parquet"),
       (1 to 10).map(i => Row(i, s"str$i")) ++ (1 to 4).map(i => Row(i, s"str$i"))
     )
-    sql("DROP TABLE IF EXISTS test_insert_parquet")
+    dropTables("test_insert_parquet")
   }
 
   test("scan a parquet table created through a CTAS statement") {
-    sql(
-      """
-        |create table test_parquet_ctas ROW FORMAT
-        |SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
-        |STORED AS
-        |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
-        |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
-        |AS select * from jt
-      """.stripMargin)
+    withTable("test_parquet_ctas") {
+      sql(
+        """
+          |create table test_parquet_ctas ROW FORMAT
+          |SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+          |STORED AS
+          |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+          |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+          |AS select * from jt
+        """.stripMargin)
 
-    checkAnswer(
-      sql(s"SELECT a, b FROM test_parquet_ctas WHERE a = 1"),
-      Seq(Row(1, "str1"))
-    )
+      checkAnswer(
+        sql(s"SELECT a, b FROM test_parquet_ctas WHERE a = 1"),
+        Seq(Row(1, "str1"))
+      )
 
-    table("test_parquet_ctas").queryExecution.optimizedPlan match {
-      case LogicalRelation(_: ParquetRelation) => // OK
-      case _ => fail(
-        "test_parquet_ctas should be converted to " +
-          s"${classOf[ParquetRelation].getCanonicalName}")
+      table("test_parquet_ctas").queryExecution.optimizedPlan match {
+        case LogicalRelation(_: ParquetRelation) => // OK
+        case _ => fail(
+          "test_parquet_ctas should be converted to " +
+              s"${classOf[ParquetRelation].getCanonicalName }")
+      }
     }
-
-    sql("DROP TABLE IF EXISTS test_parquet_ctas")
   }
 
   test("MetastoreRelation in InsertIntoTable will be converted") {
-    sql(
-      """
-        |create table test_insert_parquet
-        |(
-        |  intField INT
-        |)
-        |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
-        |STORED AS
-        |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
-        |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
-      """.stripMargin)
+    withTable("test_insert_parquet") {
+      sql(
+        """
+          |create table test_insert_parquet
+          |(
+          |  intField INT
+          |)
+          |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+          |STORED AS
+          |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+          |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+        """.stripMargin)
+
+      val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt")
+      df.queryExecution.executedPlan match {
+        case ExecutedCommand(InsertIntoHadoopFsRelation(_: ParquetRelation, _, _)) => // OK
+        case o => fail("test_insert_parquet should be converted to a " +
+          s"${classOf[ParquetRelation].getCanonicalName} and " +
+          s"${classOf[InsertIntoDataSource].getCanonicalName} is expcted as the SparkPlan. " +
+          s"However, found a ${o.toString} ")
+      }
 
-    val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt")
-    df.queryExecution.executedPlan match {
-      case ExecutedCommand(InsertIntoHadoopFsRelation(_: ParquetRelation, _, _)) => // OK
-      case o => fail("test_insert_parquet should be converted to a " +
-        s"${classOf[ParquetRelation].getCanonicalName} and " +
-        s"${classOf[InsertIntoDataSource].getCanonicalName} is expcted as the SparkPlan. " +
-        s"However, found a ${o.toString} ")
+      checkAnswer(
+        sql("SELECT intField FROM test_insert_parquet WHERE test_insert_parquet.intField > 5"),
+        sql("SELECT a FROM jt WHERE jt.a > 5").collect()
+      )
     }
-
-    checkAnswer(
-      sql("SELECT intField FROM test_insert_parquet WHERE test_insert_parquet.intField > 5"),
-      sql("SELECT a FROM jt WHERE jt.a > 5").collect()
-    )
-
-    sql("DROP TABLE IF EXISTS test_insert_parquet")
   }
 
   test("MetastoreRelation in InsertIntoHiveTable will be converted") {
-    sql(
-      """
-        |create table test_insert_parquet
-        |(
-        |  int_array array<int>
-        |)
-        |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
-        |STORED AS
-        |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
-        |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
-      """.stripMargin)
+    withTable("test_insert_parquet") {
+      sql(
+        """
+          |create table test_insert_parquet
+          |(
+          |  int_array array<int>
+          |)
+          |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+          |STORED AS
+          |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+          |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+        """.stripMargin)
+
+      val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array")
+      df.queryExecution.executedPlan match {
+        case ExecutedCommand(InsertIntoHadoopFsRelation(r: ParquetRelation, _, _)) => // OK
+        case o => fail("test_insert_parquet should be converted to a " +
+          s"${classOf[ParquetRelation].getCanonicalName} and " +
+          s"${classOf[InsertIntoDataSource].getCanonicalName} is expcted as the SparkPlan." +
+          s"However, found a ${o.toString} ")
+      }
 
-    val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array")
-    df.queryExecution.executedPlan match {
-      case ExecutedCommand(InsertIntoHadoopFsRelation(r: ParquetRelation, _, _)) => // OK
-      case o => fail("test_insert_parquet should be converted to a " +
-        s"${classOf[ParquetRelation].getCanonicalName} and " +
-        s"${classOf[InsertIntoDataSource].getCanonicalName} is expcted as the SparkPlan." +
-        s"However, found a ${o.toString} ")
+      checkAnswer(
+        sql("SELECT int_array FROM test_insert_parquet"),
+        sql("SELECT a FROM jt_array").collect()
+      )
     }
-
-    checkAnswer(
-      sql("SELECT int_array FROM test_insert_parquet"),
-      sql("SELECT a FROM jt_array").collect()
-    )
-
-    sql("DROP TABLE IF EXISTS test_insert_parquet")
   }
 
   test("SPARK-6450 regression test") {
-    sql(
-      """CREATE TABLE IF NOT EXISTS ms_convert (key INT)
-        |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
-        |STORED AS
-        |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
-        |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
-      """.stripMargin)
+    withTable("ms_convert") {
+      sql(
+        """CREATE TABLE IF NOT EXISTS ms_convert (key INT)
+          |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+          |STORED AS
+          |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+          |  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+        """.stripMargin)
+
+      // This shouldn't throw AnalysisException
+      val analyzed = sql(
+        """SELECT key FROM ms_convert
+          |UNION ALL
+          |SELECT key FROM ms_convert
+        """.stripMargin).queryExecution.analyzed
 
-    // This shouldn't throw AnalysisException
-    val analyzed = sql(
-      """SELECT key FROM ms_convert
-        |UNION ALL
-        |SELECT key FROM ms_convert
-      """.stripMargin).queryExecution.analyzed
-
-    assertResult(2) {
-      analyzed.collect {
-        case r @ LogicalRelation(_: ParquetRelation) => r
-      }.size
+      assertResult(2) {
+        analyzed.collect {
+          case r@LogicalRelation(_: ParquetRelation) => r
+        }.size
+      }
     }
-
-    sql("DROP TABLE ms_convert")
   }
 
   def collectParquetRelation(df: DataFrame): ParquetRelation = {
@@ -379,42 +387,42 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
   }
 
   test("SPARK-7749: non-partitioned metastore Parquet table lookup should use cached relation") {
-    sql(
-      s"""CREATE TABLE nonPartitioned (
-         |  key INT,
-         |  value STRING
-         |)
-         |STORED AS PARQUET
-       """.stripMargin)
-
-    // First lookup fills the cache
-    val r1 = collectParquetRelation(table("nonPartitioned"))
-    // Second lookup should reuse the cache
-    val r2 = collectParquetRelation(table("nonPartitioned"))
-    // They should be the same instance
-    assert(r1 eq r2)
-
-    sql("DROP TABLE nonPartitioned")
+    withTable("nonPartitioned") {
+      sql(
+        s"""CREATE TABLE nonPartitioned (
+           |  key INT,
+           |  value STRING
+           |)
+           |STORED AS PARQUET
+         """.stripMargin)
+
+      // First lookup fills the cache
+      val r1 = collectParquetRelation(table("nonPartitioned"))
+      // Second lookup should reuse the cache
+      val r2 = collectParquetRelation(table("nonPartitioned"))
+      // They should be the same instance
+      assert(r1 eq r2)
+    }
   }
 
   test("SPARK-7749: partitioned metastore Parquet table lookup should use cached relation") {
-    sql(
-      s"""CREATE TABLE partitioned (
-         |  key INT,
-         |  value STRING
-         |)
-         |PARTITIONED BY (part INT)
-         |STORED AS PARQUET
+    withTable("partitioned") {
+      sql(
+        s"""CREATE TABLE partitioned (
+           | key INT,
+           | value STRING
+           |)
+           |PARTITIONED BY (part INT)
+           |STORED AS PARQUET
        """.stripMargin)
 
-    // First lookup fills the cache
-    val r1 = collectParquetRelation(table("partitioned"))
-    // Second lookup should reuse the cache
-    val r2 = collectParquetRelation(table("partitioned"))
-    // They should be the same instance
-    assert(r1 eq r2)
-
-    sql("DROP TABLE partitioned")
+      // First lookup fills the cache
+      val r1 = collectParquetRelation(table("partitioned"))
+      // Second lookup should reuse the cache
+      val r2 = collectParquetRelation(table("partitioned"))
+      // They should be the same instance
+      assert(r1 eq r2)
+    }
   }
 
   test("Caching converted data source Parquet Relations") {
@@ -430,8 +438,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
       }
     }
 
-    sql("DROP TABLE IF EXISTS test_insert_parquet")
-    sql("DROP TABLE IF EXISTS test_parquet_partitioned_cache_test")
+    dropTables("test_insert_parquet", "test_parquet_partitioned_cache_test")
 
     sql(
       """
@@ -479,7 +486,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
         |  intField INT,
         |  stringField STRING
         |)
-        |PARTITIONED BY (date string)
+        |PARTITIONED BY (`date` string)
         |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
         |STORED AS
         |  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
@@ -491,7 +498,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
     sql(
       """
         |INSERT INTO TABLE test_parquet_partitioned_cache_test
-        |PARTITION (date='2015-04-01')
+        |PARTITION (`date`='2015-04-01')
         |select a, b from jt
       """.stripMargin)
     // Right now, insert into a partitioned Parquet is not supported in data source Parquet.
@@ -500,7 +507,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
     sql(
       """
         |INSERT INTO TABLE test_parquet_partitioned_cache_test
-        |PARTITION (date='2015-04-02')
+        |PARTITION (`date`='2015-04-02')
         |select a, b from jt
       """.stripMargin)
     assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null)
@@ -510,7 +517,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
     checkCached(tableIdentifier)
     // Make sure we can read the data.
     checkAnswer(
-      sql("select STRINGField, date, intField from test_parquet_partitioned_cache_test"),
+      sql("select STRINGField, `date`, intField from test_parquet_partitioned_cache_test"),
       sql(
         """
           |select b, '2015-04-01', a FROM jt
@@ -521,8 +528,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
     invalidateTable("test_parquet_partitioned_cache_test")
     assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null)
 
-    sql("DROP TABLE test_insert_parquet")
-    sql("DROP TABLE test_parquet_partitioned_cache_test")
+    dropTables("test_insert_parquet", "test_parquet_partitioned_cache_test")
   }
 }
 
@@ -532,6 +538,11 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest {
 class ParquetSourceSuite extends ParquetPartitioningTest {
   override def beforeAll(): Unit = {
     super.beforeAll()
+    dropTables("partitioned_parquet",
+      "partitioned_parquet_with_key",
+      "partitioned_parquet_with_complextypes",
+      "partitioned_parquet_with_key_and_complextypes",
+      "normal_parquet")
 
     sql( s"""
       create temporary table partitioned_parquet
@@ -635,22 +646,22 @@ class ParquetSourceSuite extends ParquetPartitioningTest {
           StructField("a", arrayType1, nullable = true) :: Nil)
     assert(df.schema === expectedSchema1)
 
-    df.write.format("parquet").saveAsTable("alwaysNullable")
+    withTable("alwaysNullable") {
+      df.write.format("parquet").saveAsTable("alwaysNullable")
 
-    val mapType2 = MapType(IntegerType, IntegerType, valueContainsNull = true)
-    val arrayType2 = ArrayType(IntegerType, containsNull = true)
-    val expectedSchema2 =
-      StructType(
-        StructField("m", mapType2, nullable = true) ::
-          StructField("a", arrayType2, nullable = true) :: Nil)
+      val mapType2 = MapType(IntegerType, IntegerType, valueContainsNull = true)
+      val arrayType2 = ArrayType(IntegerType, containsNull = true)
+      val expectedSchema2 =
+        StructType(
+          StructField("m", mapType2, nullable = true) ::
+              StructField("a", arrayType2, nullable = true) :: Nil)
 
-    assert(table("alwaysNullable").schema === expectedSchema2)
-
-    checkAnswer(
-      sql("SELECT m, a FROM alwaysNullable"),
-      Row(Map(2 -> 3), Seq(4, 5, 6)))
+      assert(table("alwaysNullable").schema === expectedSchema2)
 
-    sql("DROP TABLE alwaysNullable")
+      checkAnswer(
+        sql("SELECT m, a FROM alwaysNullable"),
+        Row(Map(2 -> 3), Seq(4, 5, 6)))
+    }
   }
 
   test("Aggregation attribute names can't contain special chars \" ,;{}()\\n\\t=\"") {
@@ -738,6 +749,16 @@ abstract class ParquetPartitioningTest extends QueryTest with SQLTestUtils with
     partitionedTableDirWithKeyAndComplexTypes.delete()
   }
 
+  /**
+   * Drop named tables if they exist
+   * @param tableNames tables to drop
+   */
+  def dropTables(tableNames: String*): Unit = {
+    tableNames.foreach { name =>
+      sql(s"DROP TABLE IF EXISTS $name")
+    }
+  }
+
   Seq(
     "partitioned_parquet",
     "partitioned_parquet_with_key",
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 2aeed98285aa86efc91b1c5b46c086358bc7ef5c..49360c48256eabab097979eeeb0573ff81c59532 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -30,7 +30,6 @@
   <name>Spark Project YARN</name>
   <properties>
     <sbt.project.name>yarn</sbt.project.name>
-    <jersey.version>1.9</jersey.version>
   </properties>
 
   <dependencies>
@@ -125,25 +124,16 @@
      <dependency>
        <groupId>com.sun.jersey</groupId>
        <artifactId>jersey-core</artifactId>
-       <version>${jersey.version}</version>
        <scope>test</scope>
      </dependency>
      <dependency>
        <groupId>com.sun.jersey</groupId>
        <artifactId>jersey-json</artifactId>
-       <version>${jersey.version}</version>
        <scope>test</scope>
-       <exclusions>
-        <exclusion>
-          <groupId>stax</groupId>
-          <artifactId>stax-api</artifactId>
-        </exclusion>
-       </exclusions>
      </dependency>
      <dependency>
        <groupId>com.sun.jersey</groupId>
        <artifactId>jersey-server</artifactId>
-       <version>${jersey.version}</version>
        <scope>test</scope>
      </dependency>
   </dependencies>
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 547863d9a07399744cf5f8868ce63b2991c79014..eb6e1fd37062012d05980ac0c953787b55caf150 100644
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -384,19 +384,29 @@ private object YarnClusterDriver extends Logging with Matchers {
 
 }
 
-private object YarnClasspathTest {
+private object YarnClasspathTest extends Logging {
+
+  var exitCode = 0
+
+  def error(m: String, ex: Throwable = null): Unit = {
+    logError(m, ex)
+    // scalastyle:off println
+    System.out.println(m)
+    if (ex != null) {
+      ex.printStackTrace(System.out)
+    }
+    // scalastyle:on println
+  }
 
   def main(args: Array[String]): Unit = {
     if (args.length != 2) {
-      // scalastyle:off println
-      System.err.println(
+      error(
         s"""
         |Invalid command line: ${args.mkString(" ")}
         |
         |Usage: YarnClasspathTest [driver result file] [executor result file]
         """.stripMargin)
       // scalastyle:on println
-      System.exit(1)
     }
 
     readResource(args(0))
@@ -406,6 +416,7 @@ private object YarnClasspathTest {
     } finally {
       sc.stop()
     }
+    System.exit(exitCode)
   }
 
   private def readResource(resultPath: String): Unit = {
@@ -415,6 +426,11 @@ private object YarnClasspathTest {
       val resource = ccl.getResourceAsStream("test.resource")
       val bytes = ByteStreams.toByteArray(resource)
       result = new String(bytes, 0, bytes.length, UTF_8)
+    } catch {
+      case t: Throwable =>
+        error(s"loading test.resource to $resultPath", t)
+        // set the exit code if not yet set
+        exitCode = 2
     } finally {
       Files.write(result, new File(resultPath), UTF_8)
     }