[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-14 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/17232


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-14 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r106048318
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala ---
@@ -898,3 +924,107 @@ private[client] class Shim_v2_0 extends Shim_v1_2 {
   }
 
 }
+
+private[client] class Shim_v2_1 extends Shim_v2_0 {
+  private lazy val loadPartitionMethod =
+findMethod(
+  classOf[Hive],
+  "loadPartition",
+  classOf[Path],
+  classOf[String],
+  classOf[JMap[String, String]],
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE)
+  private lazy val loadTableMethod =
+findMethod(
+  classOf[Hive],
+  "loadTable",
+  classOf[Path],
+  classOf[String],
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE)
+  private lazy val loadDynamicPartitionsMethod =
+findMethod(
+  classOf[Hive],
+  "loadDynamicPartitions",
+  classOf[Path],
+  classOf[String],
+  classOf[JMap[String, String]],
+  JBoolean.TYPE,
+  JInteger.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JLong.TYPE,
+  JBoolean.TYPE,
+  classOf[AcidUtils.Operation])
+  private lazy val alterTableMethod =
+findMethod(
+  classOf[Hive],
+  "alterTable",
+  classOf[String],
+  classOf[Table],
+  classOf[EnvironmentContext])
+  private lazy val alterPartitionsMethod =
+findMethod(
+  classOf[Hive],
+  "alterPartitions",
+  classOf[String],
+  classOf[JList[Partition]],
+  classOf[EnvironmentContext])
+
+  override def loadPartition(
+  hive: Hive,
+  loadPath: Path,
+  tableName: String,
+  partSpec: JMap[String, String],
+  replace: Boolean,
+  inheritTableSpecs: Boolean,
+  isSkewedStoreAsSubdir: Boolean,
+  isSrcLocal: Boolean): Unit = {
+loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, 
replace: JBoolean,
+  inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean,
+  isSrcLocal: JBoolean, JBoolean.FALSE, JBoolean.FALSE)
--- End diff --

Agree. Let me fix all of them. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-14 Thread sameeragarwal
Github user sameeragarwal commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r106045626
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala ---
@@ -898,3 +924,107 @@ private[client] class Shim_v2_0 extends Shim_v1_2 {
   }
 
 }
+
+private[client] class Shim_v2_1 extends Shim_v2_0 {
+  private lazy val loadPartitionMethod =
+findMethod(
+  classOf[Hive],
+  "loadPartition",
+  classOf[Path],
+  classOf[String],
+  classOf[JMap[String, String]],
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE)
+  private lazy val loadTableMethod =
+findMethod(
+  classOf[Hive],
+  "loadTable",
+  classOf[Path],
+  classOf[String],
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE)
+  private lazy val loadDynamicPartitionsMethod =
+findMethod(
+  classOf[Hive],
+  "loadDynamicPartitions",
+  classOf[Path],
+  classOf[String],
+  classOf[JMap[String, String]],
+  JBoolean.TYPE,
+  JInteger.TYPE,
+  JBoolean.TYPE,
+  JBoolean.TYPE,
+  JLong.TYPE,
+  JBoolean.TYPE,
+  classOf[AcidUtils.Operation])
+  private lazy val alterTableMethod =
+findMethod(
+  classOf[Hive],
+  "alterTable",
+  classOf[String],
+  classOf[Table],
+  classOf[EnvironmentContext])
+  private lazy val alterPartitionsMethod =
+findMethod(
+  classOf[Hive],
+  "alterPartitions",
+  classOf[String],
+  classOf[JList[Partition]],
+  classOf[EnvironmentContext])
+
+  override def loadPartition(
+  hive: Hive,
+  loadPath: Path,
+  tableName: String,
+  partSpec: JMap[String, String],
+  replace: Boolean,
+  inheritTableSpecs: Boolean,
+  isSkewedStoreAsSubdir: Boolean,
+  isSrcLocal: Boolean): Unit = {
+loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, 
replace: JBoolean,
+  inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean,
+  isSrcLocal: JBoolean, JBoolean.FALSE, JBoolean.FALSE)
--- End diff --

nit: From a readability perspective, would it make sense to explicitly 
define `isAcid` and `hasFollowingStatsTask` and set them to `JBoolean.FALSE` 
(along with some comments for the latter)? We can then use them here and in all 
the methods below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-14 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r106039359
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala ---
@@ -120,10 +121,12 @@ class VersionsSuite extends SparkFunSuite with 
Logging {
   System.gc() // Hack to avoid SEGV on some JVM versions.
   val hadoopConf = new Configuration()
   hadoopConf.set("test", "success")
-  // Hive changed the default of datanucleus.schema.autoCreateAll from 
true to false since 2.0
-  // For details, see the JIRA HIVE-6113
-  if (version == "2.0") {
+  // Hive changed the default of datanucleus.schema.autoCreateAll from 
true to false and
+  // hive.metastore.schema.verification from false to true since 2.0
+  // For details, see the JIRA HIVE-6113 and HIVE-12463
+  if (version == "2.0" || version == "2.1") {
--- End diff --

Either way is fine to me, but `datanucleus.schema.autoCreateAll` was 
introduced in Hive 2.0 with HIVE-6113 to replace 
`datanucleus.autoCreateSchema`. That is the major reason why I keep it.

Thanks for the review!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r106037364
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala ---
@@ -120,10 +121,12 @@ class VersionsSuite extends SparkFunSuite with 
Logging {
   System.gc() // Hack to avoid SEGV on some JVM versions.
   val hadoopConf = new Configuration()
   hadoopConf.set("test", "success")
-  // Hive changed the default of datanucleus.schema.autoCreateAll from 
true to false since 2.0
-  // For details, see the JIRA HIVE-6113
-  if (version == "2.0") {
+  // Hive changed the default of datanucleus.schema.autoCreateAll from 
true to false and
+  // hive.metastore.schema.verification from false to true since 2.0
+  // For details, see the JIRA HIVE-6113 and HIVE-12463
+  if (version == "2.0" || version == "2.1") {
--- End diff --

Is there an issue with setting these values regardless of the version? 
Adding more and more versions to this whitelist feels sub-optimal.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r105551943
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -94,6 +94,10 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 try {
   body
 } catch {
+  case i: InvocationTargetException if 
isClientException(i.getTargetException) =>
+val e = i.getTargetException
+throw new AnalysisException(
+  e.getClass.getCanonicalName + ": " + e.getMessage, cause = 
Some(e))
--- End diff --

The PR is submitted https://github.com/apache/spark/pull/17265


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r105548919
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -94,6 +94,10 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 try {
   body
 } catch {
+  case i: InvocationTargetException if 
isClientException(i.getTargetException) =>
+val e = i.getTargetException
+throw new AnalysisException(
+  e.getClass.getCanonicalName + ": " + e.getMessage, cause = 
Some(e))
--- End diff --

This fix needs to be backported to the previous releases. Will submit a 
separate one for this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r105453731
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
 ---
@@ -95,6 +95,7 @@ private[hive] object IsolatedClientLoader extends Logging 
{
 case "1.1" | "1.1.0" => hive.v1_1
 case "1.2" | "1.2.0" | "1.2.1" => hive.v1_2
 case "2.0" | "2.0.0" | "2.0.1" => hive.v2_0
+case "2.1" | "2.1.0" => hive.v2_1
--- End diff --

I plan to do it later. Thus, I mark it as WIP. The PR is not ready for 
review


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-10 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r105449419
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala ---
@@ -67,7 +67,11 @@ package object client {
   exclusions = Seq("org.apache.curator:*",
 "org.pentaho:pentaho-aggdesigner-algorithm"))
 
-val allSupportedHiveVersions = Set(v12, v13, v14, v1_0, v1_1, v1_2, 
v2_0)
+case object v2_1 extends HiveVersion("2.1.0",
--- End diff --

2.1.1 here, too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-10 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/17232#discussion_r105448253
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
 ---
@@ -95,6 +95,7 @@ private[hive] object IsolatedClientLoader extends Logging 
{
 case "1.1" | "1.1.0" => hive.v1_1
 case "1.2" | "1.2.0" | "1.2.1" => hive.v1_2
 case "2.0" | "2.0.0" | "2.0.1" => hive.v2_0
+case "2.1" | "2.1.0" => hive.v2_1
--- End diff --

Hi, @gatorsmile .
It's great. Could we add "2.1.1" in this PR, too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17232: [SPARK-18112] [SQL] Support reading data from Hiv...

2017-03-09 Thread gatorsmile
GitHub user gatorsmile opened a pull request:

https://github.com/apache/spark/pull/17232

[SPARK-18112] [SQL] Support reading data from Hive 2.1 metastore [WIP]

### What changes were proposed in this pull request?
This PR is to support reading data from Hive 2.1 metastore. Need to update 
shim class because of the Hive API changes caused by the following two Hive 
JIRAs:
- [HIVE-12730 MetadataUpdater: provide a mechanism to edit the basic 
statistics of a table (or a 
partition)](https://issues.apache.org/jira/browse/HIVE-12730)
- [Hive-13341 Stats state is not captured correctly: differentiate load 
table and create table](https://issues.apache.org/jira/browse/HIVE-13341)

There two new fields have been added in Hive.  
- `EnvironmentContext environmentContext`. So far, this is always set to 
`null`. This was introduced for supporting DDL `alter table s update statistics 
set ('numRows'='NaN')`. Using this DDL, users can specify the statistics. So 
far, our Spark SQL does not need it, because we use different table properties 
to store our generated statistics values. However, when Spark SQL issues ALTER 
TABLE DDL statements, Hive metastore always automatically invalidate the 
Hive-generated statistics. In the follow-up PR, we can fix it by explicitly 
adding a property to `environmentContext`.
```JAVA
putToProperties(StatsSetupConst.STATS_GENERATED, StatsSetupConst.USER)
```
- `boolean hasFollowingStatsTask`. We always set it to `false`. TODO: more 
investigation about this

### How was this patch tested?
Added test cases to VersionsSuite.scala

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/gatorsmile/spark Hive21

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/17232.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #17232


commit af81cee9f54abc13d7d07a12e4b499e49cd0dbcb
Author: Xiao Li 
Date:   2017-03-10T03:35:38Z

fix




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org