[GitHub] spark pull request #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-02 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-02 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86273774
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala 
---
@@ -517,15 +517,15 @@ class SQLQuerySuite extends QueryTest with 
SQLTestUtils with TestHiveSingleton {
 val catalogTable =
   sessionState.catalog.getTableMetadata(TableIdentifier(tableName))
 relation match {
-  case LogicalRelation(r: HadoopFsRelation, _, _) =>
+  case LogicalRelation(r: HadoopFsRelation, _, Some(table)) =>
 if (!isDataSourceTable) {
   fail(
 s"${classOf[MetastoreRelation].getCanonicalName} is expected, 
but found " +
   s"${HadoopFsRelation.getClass.getCanonicalName}.")
 }
 userSpecifiedLocation match {
   case Some(location) =>
-assert(r.options("path") === location)
+assert(table.storage.locationUri.get === location)
--- End diff --

Seems we can revert this changes. 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-02 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86272489
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -189,66 +188,39 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   throw new TableAlreadyExistsException(db = db, table = table)
 }
 // Before saving data source table metadata into Hive metastore, we 
should:
-//  1. Put table provider, schema, partition column names, bucket 
specification and partition
-// provider in table properties.
+//  1. Put table metadata like provider, schema, etc. in table 
properties.
 //  2. Check if this table is hive compatible
-//2.1  If it's not hive compatible, set schema, partition columns 
and bucket spec to empty
-// and save table metadata to Hive.
+//2.1  If it's not hive compatible, set location URI, schema, 
partition columns and bucket
+// spec to empty and save table metadata to Hive.
 //2.2  If it's hive compatible, set serde information in table 
metadata and try to save
 // it to Hive. If it fails, treat it as not hive compatible 
and go back to 2.1
 if (DDLUtils.isDatasourceTable(tableDefinition)) {
-  // data source table always have a provider, it's guaranteed by 
`DDLUtils.isDatasourceTable`.
-  val provider = tableDefinition.provider.get
-  val partitionColumns = tableDefinition.partitionColumnNames
-  val bucketSpec = tableDefinition.bucketSpec
-
-  val tableProperties = new scala.collection.mutable.HashMap[String, 
String]
-  tableProperties.put(DATASOURCE_PROVIDER, provider)
-  if (tableDefinition.partitionProviderIsHive) {
-tableProperties.put(TABLE_PARTITION_PROVIDER, "hive")
-  }
-
-  // Serialized JSON schema string may be too long to be stored into a 
single metastore table
-  // property. In this case, we split the JSON string and store each 
part as a separate table
-  // property.
-  val threshold = conf.get(SCHEMA_STRING_LENGTH_THRESHOLD)
-  val schemaJsonString = tableDefinition.schema.json
-  // Split the JSON string.
-  val parts = schemaJsonString.grouped(threshold).toSeq
-  tableProperties.put(DATASOURCE_SCHEMA_NUMPARTS, parts.size.toString)
-  parts.zipWithIndex.foreach { case (part, index) =>
-tableProperties.put(s"$DATASOURCE_SCHEMA_PART_PREFIX$index", part)
-  }
-
-  if (partitionColumns.nonEmpty) {
-tableProperties.put(DATASOURCE_SCHEMA_NUMPARTCOLS, 
partitionColumns.length.toString)
-partitionColumns.zipWithIndex.foreach { case (partCol, index) =>
-  tableProperties.put(s"$DATASOURCE_SCHEMA_PARTCOL_PREFIX$index", 
partCol)
-}
-  }
-
-  if (bucketSpec.isDefined) {
-val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames) = 
bucketSpec.get
+  val tableProperties = tableMetaToTableProps(tableDefinition)
 
-tableProperties.put(DATASOURCE_SCHEMA_NUMBUCKETS, 
numBuckets.toString)
-tableProperties.put(DATASOURCE_SCHEMA_NUMBUCKETCOLS, 
bucketColumnNames.length.toString)
-bucketColumnNames.zipWithIndex.foreach { case (bucketCol, index) =>
-  
tableProperties.put(s"$DATASOURCE_SCHEMA_BUCKETCOL_PREFIX$index", bucketCol)
-}
-
-if (sortColumnNames.nonEmpty) {
-  tableProperties.put(DATASOURCE_SCHEMA_NUMSORTCOLS, 
sortColumnNames.length.toString)
-  sortColumnNames.zipWithIndex.foreach { case (sortCol, index) =>
-
tableProperties.put(s"$DATASOURCE_SCHEMA_SORTCOL_PREFIX$index", sortCol)
-  }
-}
+  val needDefaultTableLocation = tableDefinition.tableType == MANAGED 
&&
+tableDefinition.storage.locationUri.isEmpty
--- End diff --

Explain when the table is managed but the location uri is not empty.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-02 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86117289
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -259,10 +266,9 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 val location = if (tableDefinition.tableType == EXTERNAL) {
   // When we hit this branch, we are saving an external data 
source table with hive
   // compatible format, which means the data source is file-based 
and must have a `path`.
-  val map = new 
CaseInsensitiveMap(tableDefinition.storage.properties)
-  require(map.contains("path"),
+  require(tableDefinition.storage.locationUri.isDefined,
 "External file-based data source table must have a `path` 
entry in storage properties.")
-  Some(new Path(map("path")).toUri.toString)
+  Some(new 
Path(tableDefinition.storage.locationUri.get).toUri.toString)
--- End diff --

maybe rename it to something else? this can be any string given by users in 
path option, and may not be a uri


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-02 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86116841
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
@@ -91,7 +73,8 @@ case class CreateTableLikeCommand(
   CatalogTable(
 identifier = targetTable,
 tableType = CatalogTableType.MANAGED,
-storage = newStorage,
+// We are creating a new managed table, which should not have 
custom table location.
+storage = sourceTableDesc.storage.copy(locationUri = None),
--- End diff --

yea


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86086688
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala ---
@@ -0,0 +1,97 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.sources
+
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, SQLContext}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.StructType
+
+class TestOptionsSource extends RelationProvider with 
CreatableRelationProvider {
+
+  override def createRelation(
+  sqlContext: SQLContext,
+  parameters: Map[String, String]): BaseRelation = {
+new TestOptionsRelation(parameters)(sqlContext.sparkSession)
+  }
+
+  override def createRelation(
+  sqlContext: SQLContext,
+  mode: SaveMode,
+  parameters: Map[String, String],
+  data: DataFrame): BaseRelation = {
+new TestOptionsRelation(parameters)(sqlContext.sparkSession)
+  }
+}
+
+class TestOptionsRelation(val options: Map[String, String])(@transient val 
session: SparkSession)
+  extends BaseRelation {
+
+  override def sqlContext: SQLContext = session.sqlContext
+
+  override def schema: StructType = new StructType().add("i", "int")
+}
+
+class PathOptionSuite extends DataSourceTest with SharedSQLContext {
+
+  test("path option always exist") {
+withTable("src") {
+  sql(
+s"""
+   |CREATE TABLE src(i int)
+   |USING ${classOf[TestOptionsSource].getCanonicalName}
+   |OPTIONS (PATH '/tmp/path')""".stripMargin)
+  assert(getPathOption("src") == Some("/tmp/path"))
+}
+
+// should exist even path option is not specified when creating table
+withTable("src") {
+  sql(s"CREATE TABLE src(i int) USING 
${classOf[TestOptionsSource].getCanonicalName}")
+  assert(getPathOption("src") == Some(defaultTablePath("src")))
+}
+  }
+
+  test("path option always represent the value of table location") {
+withTable("src") {
+  sql(
+s"""
+   |CREATE TABLE src(i int)
+   |USING ${classOf[TestOptionsSource].getCanonicalName}
+   |OPTIONS (PATH '/tmp/path')""".stripMargin)
+  sql("ALTER TABLE src SET LOCATION '/tmp/path2'")
+  assert(getPathOption("src") == Some("/tmp/path2"))
+}
+
+withTable("src", "src2") {
+  sql(s"CREATE TABLE src(i int) USING 
${classOf[TestOptionsSource].getCanonicalName}")
+  sql("ALTER TABLE src RENAME TO src2")
--- End diff --

Agree. But we still need a test case to verify the code path and the 
behavior you mentioned above. So far, it sounds like we do not have any 
end-to-end test case for `RENAME TABLE` using HiveExternalCatalog. I manually 
verified it and it works.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-02 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86085817
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala ---
@@ -0,0 +1,97 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.sources
+
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, SQLContext}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.StructType
+
+class TestOptionsSource extends RelationProvider with 
CreatableRelationProvider {
+
+  override def createRelation(
+  sqlContext: SQLContext,
+  parameters: Map[String, String]): BaseRelation = {
+new TestOptionsRelation(parameters)(sqlContext.sparkSession)
+  }
+
+  override def createRelation(
+  sqlContext: SQLContext,
+  mode: SaveMode,
+  parameters: Map[String, String],
+  data: DataFrame): BaseRelation = {
+new TestOptionsRelation(parameters)(sqlContext.sparkSession)
+  }
+}
+
+class TestOptionsRelation(val options: Map[String, String])(@transient val 
session: SparkSession)
+  extends BaseRelation {
+
+  override def sqlContext: SQLContext = session.sqlContext
+
+  override def schema: StructType = new StructType().add("i", "int")
+}
+
+class PathOptionSuite extends DataSourceTest with SharedSQLContext {
+
+  test("path option always exist") {
+withTable("src") {
+  sql(
+s"""
+   |CREATE TABLE src(i int)
+   |USING ${classOf[TestOptionsSource].getCanonicalName}
+   |OPTIONS (PATH '/tmp/path')""".stripMargin)
+  assert(getPathOption("src") == Some("/tmp/path"))
+}
+
+// should exist even path option is not specified when creating table
+withTable("src") {
+  sql(s"CREATE TABLE src(i int) USING 
${classOf[TestOptionsSource].getCanonicalName}")
+  assert(getPathOption("src") == Some(defaultTablePath("src")))
+}
+  }
+
+  test("path option always represent the value of table location") {
+withTable("src") {
+  sql(
+s"""
+   |CREATE TABLE src(i int)
+   |USING ${classOf[TestOptionsSource].getCanonicalName}
+   |OPTIONS (PATH '/tmp/path')""".stripMargin)
+  sql("ALTER TABLE src SET LOCATION '/tmp/path2'")
+  assert(getPathOption("src") == Some("/tmp/path2"))
+}
+
+withTable("src", "src2") {
+  sql(s"CREATE TABLE src(i int) USING 
${classOf[TestOptionsSource].getCanonicalName}")
+  sql("ALTER TABLE src RENAME TO src2")
--- End diff --

It's nothing about `ExternalCatalog`, as `ExternalCatalog` doesn't need to 
know about the path option. We generate the path option outside of 
`ExternalCatalog`, and we only need `ExternalCatalog` to put table location in 
the `locationUri` field correctly.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86075163
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala ---
@@ -0,0 +1,97 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.sources
+
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, SQLContext}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.StructType
+
+class TestOptionsSource extends RelationProvider with 
CreatableRelationProvider {
+
+  override def createRelation(
+  sqlContext: SQLContext,
+  parameters: Map[String, String]): BaseRelation = {
+new TestOptionsRelation(parameters)(sqlContext.sparkSession)
+  }
+
+  override def createRelation(
+  sqlContext: SQLContext,
+  mode: SaveMode,
+  parameters: Map[String, String],
+  data: DataFrame): BaseRelation = {
+new TestOptionsRelation(parameters)(sqlContext.sparkSession)
+  }
+}
+
+class TestOptionsRelation(val options: Map[String, String])(@transient val 
session: SparkSession)
+  extends BaseRelation {
+
+  override def sqlContext: SQLContext = session.sqlContext
+
+  override def schema: StructType = new StructType().add("i", "int")
+}
+
+class PathOptionSuite extends DataSourceTest with SharedSQLContext {
+
+  test("path option always exist") {
+withTable("src") {
+  sql(
+s"""
+   |CREATE TABLE src(i int)
+   |USING ${classOf[TestOptionsSource].getCanonicalName}
+   |OPTIONS (PATH '/tmp/path')""".stripMargin)
+  assert(getPathOption("src") == Some("/tmp/path"))
+}
+
+// should exist even path option is not specified when creating table
+withTable("src") {
+  sql(s"CREATE TABLE src(i int) USING 
${classOf[TestOptionsSource].getCanonicalName}")
+  assert(getPathOption("src") == Some(defaultTablePath("src")))
+}
+  }
+
+  test("path option always represent the value of table location") {
+withTable("src") {
+  sql(
+s"""
+   |CREATE TABLE src(i int)
+   |USING ${classOf[TestOptionsSource].getCanonicalName}
+   |OPTIONS (PATH '/tmp/path')""".stripMargin)
+  sql("ALTER TABLE src SET LOCATION '/tmp/path2'")
+  assert(getPathOption("src") == Some("/tmp/path2"))
+}
+
+withTable("src", "src2") {
+  sql(s"CREATE TABLE src(i int) USING 
${classOf[TestOptionsSource].getCanonicalName}")
+  sql("ALTER TABLE src RENAME TO src2")
--- End diff --

This test case is still calling the `InMemoryCatalog.renameTable`. Thus, we 
still need a test case to verify the behavior of 
`HiveExternalCatalog.renameTable`


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86070024
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
@@ -91,7 +73,8 @@ case class CreateTableLikeCommand(
   CatalogTable(
 identifier = targetTable,
 tableType = CatalogTableType.MANAGED,
-storage = newStorage,
+// We are creating a new managed table, which should not have 
custom table location.
+storage = sourceTableDesc.storage.copy(locationUri = None),
--- End diff --

When will we set the location? Is it set by hive metastore?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86070455
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -207,6 +207,9 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   if (tableDefinition.partitionProviderIsHive) {
 tableProperties.put(TABLE_PARTITION_PROVIDER, "hive")
   }
+  tableDefinition.storage.locationUri.foreach { location =>
+tableProperties.put(TABLE_LOCATION, location)
--- End diff --

why do we need this? Why not just use `path` in serde properties?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86069832
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -85,14 +86,7 @@ case class CreateDataSourceTableCommand(table: 
CatalogTable, ignoreIfExists: Boo
   }
 }
 
-val optionsWithPath = if (table.tableType == CatalogTableType.MANAGED) 
{
-  table.storage.properties + ("path" -> 
sessionState.catalog.defaultTablePath(table.identifier))
--- End diff --

where do we assign the default location?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86069964
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -665,15 +665,7 @@ case class AlterTableSetLocationCommand(
 catalog.alterPartitions(tableName, Seq(newPart))
   case None =>
 // No partition spec is specified, so we set the location for the 
table itself
-val newTable =
-  if (DDLUtils.isDatasourceTable(table)) {
-table.withNewStorage(
-  locationUri = Some(location),
-  properties = table.storage.properties ++ Map("path" -> 
location))
-  } else {
-table.withNewStorage(locationUri = Some(location))
-  }
-catalog.alterTable(newTable)
+catalog.alterTable(table.withNewStorage(locationUri = 
Some(location)))
--- End diff --

Do we still have this issue?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86070624
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -383,8 +389,22 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   }
 
   override def renameTable(db: String, oldName: String, newName: String): 
Unit = withClient {
-val newTable = client.getTable(db, oldName)
-  .copy(identifier = TableIdentifier(newName, Some(db)))
+val rawTable = client.getTable(db, oldName)
+
+val tableProps = if (rawTable.tableType == MANAGED) {
+  // If it's a managed table and we are renaming it, then the 
TABLE_LOCATION property becomes
+  // inaccurate as Hive metastore will generate a new table location 
in the `locationUri` field.
+  // Here we remove the TABLE_LOCATION property, so that we can read 
the value of `locationUri`
+  // field and treat it as table location when we read this table 
later.
+  rawTable.properties - TABLE_LOCATION
+} else {
+  rawTable.properties
+}
+
+val newTable = rawTable.copy(
+  identifier = TableIdentifier(newName, Some(db)),
+  properties = tableProps)
+
--- End diff --

I am not sure if I am following at here. So, after rename, we will not have 
a table property representing the location?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86070959
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -513,6 +555,16 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 tableWithStats.copy(properties = getOriginalTableProperties(table))
   }
 
+  private def getLocationFromRawTable(rawTable: CatalogTable): 
Option[String] = {
+rawTable.properties.get(TABLE_LOCATION).orElse {
+  // In older version of spark, we store the table location in storage 
properties with key
+  // `path`, instead of table properties with key 
`spark.sql.tableLocation`. We should
--- End diff --

Why do we need `spark.sql.tableLocation` instead of just relying on hive's 
location field and path in serde properties?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86070329
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala ---
@@ -0,0 +1,97 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.sources
+
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, SQLContext}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.StructType
+
+class TestOptionsSource extends RelationProvider with 
CreatableRelationProvider {
+
+  override def createRelation(
+  sqlContext: SQLContext,
+  parameters: Map[String, String]): BaseRelation = {
+new TestOptionsRelation(parameters)(sqlContext.sparkSession)
--- End diff --

Can we also add comment to explain which tests exercise this method?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86070568
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -259,10 +266,9 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 val location = if (tableDefinition.tableType == EXTERNAL) {
   // When we hit this branch, we are saving an external data 
source table with hive
   // compatible format, which means the data source is file-based 
and must have a `path`.
-  val map = new 
CaseInsensitiveMap(tableDefinition.storage.properties)
-  require(map.contains("path"),
+  require(tableDefinition.storage.locationUri.isDefined,
 "External file-based data source table must have a `path` 
entry in storage properties.")
-  Some(new Path(map("path")).toUri.toString)
+  Some(new 
Path(tableDefinition.storage.locationUri.get).toUri.toString)
--- End diff --

This part looks weird since we already have a location uri.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86070169
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -541,3 +434,123 @@ case class DataSource(
 }
   }
 }
+
+object DataSource {
+
+  /** A map to maintain backward compatibility in case we move data 
sources around. */
+  private val backwardCompatibilityMap: Map[String, String] = {
+val jdbc = classOf[JdbcRelationProvider].getCanonicalName
+val json = classOf[JsonFileFormat].getCanonicalName
+val parquet = classOf[ParquetFileFormat].getCanonicalName
+val csv = classOf[CSVFileFormat].getCanonicalName
+val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat"
+val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat"
+
+Map(
+  "org.apache.spark.sql.jdbc" -> jdbc,
+  "org.apache.spark.sql.jdbc.DefaultSource" -> jdbc,
+  "org.apache.spark.sql.execution.datasources.jdbc.DefaultSource" -> 
jdbc,
+  "org.apache.spark.sql.execution.datasources.jdbc" -> jdbc,
+  "org.apache.spark.sql.json" -> json,
+  "org.apache.spark.sql.json.DefaultSource" -> json,
+  "org.apache.spark.sql.execution.datasources.json" -> json,
+  "org.apache.spark.sql.execution.datasources.json.DefaultSource" -> 
json,
+  "org.apache.spark.sql.parquet" -> parquet,
+  "org.apache.spark.sql.parquet.DefaultSource" -> parquet,
+  "org.apache.spark.sql.execution.datasources.parquet" -> parquet,
+  "org.apache.spark.sql.execution.datasources.parquet.DefaultSource" 
-> parquet,
+  "org.apache.spark.sql.hive.orc.DefaultSource" -> orc,
+  "org.apache.spark.sql.hive.orc" -> orc,
+  "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm,
+  "org.apache.spark.ml.source.libsvm" -> libsvm,
+  "com.databricks.spark.csv" -> csv
+)
+  }
+
+  /**
+   * Class that were removed in Spark 2.0. Used to detect incompatibility 
libraries for Spark 2.0.
+   */
+  private val spark2RemovedClasses = Set(
+"org.apache.spark.sql.DataFrame",
+"org.apache.spark.sql.sources.HadoopFsRelationProvider",
+"org.apache.spark.Logging")
+
+  /** Given a provider name, look up the data source class definition. */
+  def lookupDataSource(provider0: String): Class[_] = {
--- End diff --

why not just provider?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86066888
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -196,18 +196,32 @@ class InMemoryCatalog(
 throw new TableAlreadyExistsException(db = db, table = table)
   }
 } else {
-  if (tableDefinition.tableType == CatalogTableType.MANAGED) {
-val dir = new Path(catalog(db).db.locationUri, table)
+  // Set the default table location if this is a managed table and its 
location is not
+  // specified.
+  // Ideally we should not create a managed table with location, but 
Hive serde table can
+  // specify location for managed table. And in 
[[CreateDataSourceTableAsSelectCommand]] we have
+  // to create the table directory and write out data before we create 
this table, to avoid
+  // exposing a partial written table.
+  val needDefaultTableLocation =
+  tableDefinition.tableType == CatalogTableType.MANAGED &&
--- End diff --

indentation


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r86065681
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -189,11 +189,11 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   throw new TableAlreadyExistsException(db = db, table = table)
 }
 // Before saving data source table metadata into Hive metastore, we 
should:
-//  1. Put table provider, schema, partition column names, bucket 
specification and partition
-// provider in table properties.
+//  1. Put table provider, location URI, schema, partition column 
names, bucket specification
+// and partition provider in table properties.
--- End diff --

Could we create a separate function for [this 
step](https://github.com/cloud-fan/spark/blob/97f04cf246e70e411a2693232e5a54f7542a17f2/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala#L206-L248)?
 The related codes become longer and longer. 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-11-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85888109
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -418,21 +424,41 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(SPARK_SQL_PREFIX))
+  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+
+  // Always update the location property w.r.t. the new table location.
+  val locationProp = tableDefinition.storage.locationUri.map { 
location =>
+TABLE_LOCATION -> location
+  }
+  // Only update the `locationUri` field if the location is really 
changed, because this table
+  // may be not Hive-compatible and can not set the `locationUri` 
field. We should respect the
+  // old `locationUri` even it's None.
+  val oldLocation = getLocationFromRawTable(oldTableDef)
+  val locationUri = if (oldLocation == 
tableDefinition.storage.locationUri) {
--- End diff --

good catch! I have fixed it in `HiveExternalCatalog.renameTable`


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85808919
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -418,21 +424,41 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(SPARK_SQL_PREFIX))
+  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+
+  // Always update the location property w.r.t. the new table location.
+  val locationProp = tableDefinition.storage.locationUri.map { 
location =>
+TABLE_LOCATION -> location
+  }
+  // Only update the `locationUri` field if the location is really 
changed, because this table
+  // may be not Hive-compatible and can not set the `locationUri` 
field. We should respect the
+  // old `locationUri` even it's None.
+  val oldLocation = getLocationFromRawTable(oldTableDef)
+  val locationUri = if (oldLocation == 
tableDefinition.storage.locationUri) {
--- End diff --

```Scala
  test("alter table - rename") {
val tabName = "tab1"
val newTabName = "tab2"
withTable(tabName, newTabName) {
  spark.range(10).write.saveAsTable(tabName)
  val catalog = spark.sessionState.catalog
  sql(s"ALTER TABLE $tabName RENAME TO $newTabName")
  sql(s"DESC FORMATTED $newTabName").show(100, false)
  assert(!catalog.tableExists(TableIdentifier(tabName)))
  assert(catalog.tableExists(TableIdentifier(newTabName)))
}
  }
```

You can try to run the above test case in `DDLSuite.scala` and 
`HiveDDLSuite.scala`. The locations are different. One is using the new table 
name; another is using the old one. 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-30 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85654615
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -784,6 +819,7 @@ object HiveExternalCatalog {
   val STATISTICS_COL_STATS_PREFIX = STATISTICS_PREFIX + "colStats."
 
   val TABLE_PARTITION_PROVIDER = SPARK_SQL_PREFIX + "partitionProvider"
+  val TABLE_LOCATION = SPARK_SQL_PREFIX + "tableLocation"
--- End diff --

I don't have a strong preference.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85624636
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -418,21 +424,41 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(SPARK_SQL_PREFIX))
+  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+
+  // Always update the location property w.r.t. the new table location.
+  val locationProp = tableDefinition.storage.locationUri.map { 
location =>
+TABLE_LOCATION -> location
+  }
+  // Only update the `locationUri` field if the location is really 
changed, because this table
+  // may be not Hive-compatible and can not set the `locationUri` 
field. We should respect the
+  // old `locationUri` even it's None.
+  val oldLocation = getLocationFromRawTable(oldTableDef)
+  val locationUri = if (oldLocation == 
tableDefinition.storage.locationUri) {
--- End diff --

When we using in-memory catalog, the location is changed for `renameTable` 
on managed data source tables. 

It sounds like we are not having the same behavior when using hive external 
catalog? 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85620456
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -784,6 +819,7 @@ object HiveExternalCatalog {
   val STATISTICS_COL_STATS_PREFIX = STATISTICS_PREFIX + "colStats."
 
   val TABLE_PARTITION_PROVIDER = SPARK_SQL_PREFIX + "partitionProvider"
+  val TABLE_LOCATION = SPARK_SQL_PREFIX + "tableLocation"
--- End diff --

Should we use `DATASOURCE_PREFIX` here? 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85488796
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -415,16 +422,34 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+
+  val oldLocation = getLocationFromRawTable(oldTableDef)
+  // Only update the `locationUri` field if the location is really 
changed, because this table
+  // may be not Hive-compatible and can not set the `locationUri` 
field. We should respect the
+  // old `locationUri` even it's None.
+  val newLocation = if (oldLocation == 
tableDefinition.storage.locationUri) {
+oldTableDef.storage.locationUri
+  } else {
+tableDefinition.storage.locationUri
+  }
--- End diff --

Shall we make `HiveDDLSuite` extends `DDLSuite`? then all these tests will 
be run with hive support automatically.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85280600
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -415,16 +422,34 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+
+  val oldLocation = getLocationFromRawTable(oldTableDef)
+  // Only update the `locationUri` field if the location is really 
changed, because this table
+  // may be not Hive-compatible and can not set the `locationUri` 
field. We should respect the
+  // old `locationUri` even it's None.
+  val newLocation = if (oldLocation == 
tableDefinition.storage.locationUri) {
+oldTableDef.storage.locationUri
+  } else {
+tableDefinition.storage.locationUri
+  }
--- End diff --

Just FYI, we do not have any test case to cover this. `ALTER TABLE SET 
LOCATION` for data source tables is only tested in `DDLSuite` (when Hive 
support is not enabled). That means, these code changes are not tested. 
Actually, this is not the only one. : (


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-26 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85239693
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -62,6 +62,7 @@ case class CreateDataSourceTableCommand(table: 
CatalogTable, ignoreIfExists: Boo
 sparkSession = sparkSession,
 userSpecifiedSchema = if (table.schema.isEmpty) None else 
Some(table.schema),
 className = table.provider.get,
+paths = table.storage.locationUri.toSeq,
 bucketSpec = table.bucketSpec,
 options = table.storage.properties).resolveRelation()
--- End diff --

Thank you! The rule looks good to me. I will try to find holes, if existed, 
tonight. : ) 

It sounds like we need test cases to verify the rules. Could you add test 
cases if @yhuai @ericl also agree on the proposal? 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-26 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85079470
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -62,6 +62,7 @@ case class CreateDataSourceTableCommand(table: 
CatalogTable, ignoreIfExists: Boo
 sparkSession = sparkSession,
 userSpecifiedSchema = if (table.schema.isEmpty) None else 
Some(table.schema),
 className = table.provider.get,
+paths = table.storage.locationUri.toSeq,
 bucketSpec = table.bucketSpec,
 options = table.storage.properties).resolveRelation()
--- End diff --

I have fixed it, with a better semantic.

Previously, although we keep the `path` option, it changes after some table 
operations, e.g. `SET LOCATION`, `RENAME TABLE`, so actually we can't use 
`path` as data source options, as it may get changed unexpectedly.

Now, we've decoupled the `path` option and table location. We infer the 
table location from `path` option at the beginning, then they are just 2 
different fields, `SET LOCATION` won't affect the `path` option but only table 
location. I have updated the PR description with detailed rules about it.

One drawback is, I have to change the semantic of `DataSource.options` a 
little. The `path` option should only take effect if the `paths` is empty. This 
means, `reader.option("path", path1).parquet(path2, path3)` will break as the 
`path1` is ignored. However, I don't think this is a reasonable use case and it 
seems fine to break it.

cc @yhuai




---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-26 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85070648
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -724,6 +758,7 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 object HiveExternalCatalog {
   val DATASOURCE_PREFIX = "spark.sql.sources."
   val DATASOURCE_PROVIDER = DATASOURCE_PREFIX + "provider"
+  val DATASOURCE_TABLE_LOCATION = DATASOURCE_PREFIX + "tableLocation"
--- End diff --

cc @yhuai we don't have to use `path` here, as it's stored in table 
properties, not storage properties. When we write the table location, we only 
write it to table properties, but when we read it, we should try table 
properties first, then storage properties, for backward compatibility.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-25 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85046747
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -741,16 +762,20 @@ object HiveExternalCatalog {
   val STATISTICS_NUM_ROWS = STATISTICS_PREFIX + "numRows"
   val STATISTICS_COL_STATS_PREFIX = STATISTICS_PREFIX + "colStats."
 
-  def removeStatsProperties(metadata: CatalogTable): Map[String, String] = 
{
-metadata.properties.filterNot { case (key, _) => 
key.startsWith(STATISTICS_PREFIX) }
+  // Ideally we should use `spark.sql.sources.location` to store the table 
location, but as we have
+  // already used `path` to store it, we should keep it for backward 
compatibility.
+  val TABLE_LOCATION = "path"
--- End diff --

the `path` option has special meaning(table location) only when it's used 
to create data source tables. Other places like the streaming code path may not 
have this semantic.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-25 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r85007038
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -62,6 +62,7 @@ case class CreateDataSourceTableCommand(table: 
CatalogTable, ignoreIfExists: Boo
 sparkSession = sparkSession,
 userSpecifiedSchema = if (table.schema.isEmpty) None else 
Some(table.schema),
 className = table.provider.get,
+paths = table.storage.locationUri.toSeq,
 bucketSpec = table.bucketSpec,
 options = table.storage.properties).resolveRelation()
--- End diff --

Here, the options do not have `path`, right? If so, the external data 
source connector will not be able to get the value of `path`. This might break 
the existing connectors. 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-25 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r84984048
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -196,18 +196,30 @@ class InMemoryCatalog(
 throw new TableAlreadyExistsException(db = db, table = table)
   }
 } else {
-  if (tableDefinition.tableType == CatalogTableType.MANAGED) {
-val dir = new Path(catalog(db).db.locationUri, table)
+  // Set the default table location if this is a managed table and its 
location is not
+  // specified.
+  // Ideally we should not create a managed table with location, but 
Hive serde table can
+  // specify location for managed table. And in 
[[CreateDataSourceTableAsSelectCommand]] we have
+  // to create the table directory and write out data before we create 
this table, to avoid
+  // exposing a partial written table.
+  val needDefaultTableLocation =
+tableDefinition.tableType == CatalogTableType.MANAGED &&
+  tableDefinition.storage.locationUri.isEmpty
--- End diff --

For a managed table, when will its location uri be empty? When it is a data 
source table?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-25 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r84976357
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -741,16 +762,20 @@ object HiveExternalCatalog {
   val STATISTICS_NUM_ROWS = STATISTICS_PREFIX + "numRows"
   val STATISTICS_COL_STATS_PREFIX = STATISTICS_PREFIX + "colStats."
 
-  def removeStatsProperties(metadata: CatalogTable): Map[String, String] = 
{
-metadata.properties.filterNot { case (key, _) => 
key.startsWith(STATISTICS_PREFIX) }
+  // Ideally we should use `spark.sql.sources.location` to store the table 
location, but as we have
+  // already used `path` to store it, we should keep it for backward 
compatibility.
+  val TABLE_LOCATION = "path"
--- End diff --

We still have the "path" literal hard-coded in several places in different 
files. Should this be pulled out into a common constant somewhere?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-25 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r84976703
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -531,3 +424,118 @@ case class DataSource(
 }
   }
 }
+
+object DataSource {
+
+  /** A map to maintain backward compatibility in case we move data 
sources around. */
+  private val backwardCompatibilityMap: Map[String, String] = {
+val jdbc = classOf[JdbcRelationProvider].getCanonicalName
+val json = classOf[JsonFileFormat].getCanonicalName
+val parquet = classOf[ParquetFileFormat].getCanonicalName
+val csv = classOf[CSVFileFormat].getCanonicalName
+val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat"
+val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat"
+
+Map(
+  "org.apache.spark.sql.jdbc" -> jdbc,
+  "org.apache.spark.sql.jdbc.DefaultSource" -> jdbc,
+  "org.apache.spark.sql.execution.datasources.jdbc.DefaultSource" -> 
jdbc,
+  "org.apache.spark.sql.execution.datasources.jdbc" -> jdbc,
+  "org.apache.spark.sql.json" -> json,
+  "org.apache.spark.sql.json.DefaultSource" -> json,
+  "org.apache.spark.sql.execution.datasources.json" -> json,
+  "org.apache.spark.sql.execution.datasources.json.DefaultSource" -> 
json,
+  "org.apache.spark.sql.parquet" -> parquet,
+  "org.apache.spark.sql.parquet.DefaultSource" -> parquet,
+  "org.apache.spark.sql.execution.datasources.parquet" -> parquet,
+  "org.apache.spark.sql.execution.datasources.parquet.DefaultSource" 
-> parquet,
+  "org.apache.spark.sql.hive.orc.DefaultSource" -> orc,
+  "org.apache.spark.sql.hive.orc" -> orc,
+  "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm,
+  "org.apache.spark.ml.source.libsvm" -> libsvm,
+  "com.databricks.spark.csv" -> csv
+)
+  }
+
+  /**
+   * Class that were removed in Spark 2.0. Used to detect incompatibility 
libraries for Spark 2.0.
+   */
+  private val spark2RemovedClasses = Set(
+"org.apache.spark.sql.DataFrame",
+"org.apache.spark.sql.sources.HadoopFsRelationProvider",
+"org.apache.spark.Logging")
+
+  /** Given a provider name, look up the data source class definition. */
+  def lookupDataSource(provider0: String): Class[_] = {
+val provider = backwardCompatibilityMap.getOrElse(provider0, provider0)
+val provider2 = s"$provider.DefaultSource"
+val loader = Utils.getContextOrSparkClassLoader
+val serviceLoader = ServiceLoader.load(classOf[DataSourceRegister], 
loader)
+
+try {
+  
serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList 
match {
+// the provider format did not match any given registered aliases
+case Nil =>
+  try {
+
Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match {
+  case Success(dataSource) =>
+// Found the data source using fully qualified path
+dataSource
+  case Failure(error) =>
+if (provider.toLowerCase == "orc" ||
+  provider.startsWith("org.apache.spark.sql.hive.orc")) {
+  throw new AnalysisException(
+"The ORC data source must be used with Hive support 
enabled")
+} else if (provider.toLowerCase == "avro" ||
+  provider == "com.databricks.spark.avro") {
+  throw new AnalysisException(
+s"Failed to find data source: ${provider.toLowerCase}. 
Please find an Avro " +
+  "package at " +
+  
"https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects;)
+} else {
+  throw new ClassNotFoundException(
+s"Failed to find data source: $provider. Please find 
packages at " +
+  
"https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects;,
+error)
+}
+}
+  } catch {
+case e: NoClassDefFoundError => // This one won't be caught by 
Scala NonFatal
+  // NoClassDefFoundError's class name uses "/" rather than 
"." for packages
+  val className = e.getMessage.replaceAll("/", ".")
+  if (spark2RemovedClasses.contains(className)) {
+throw new ClassNotFoundException(s"$className was removed 
in Spark 2.0. " +
+  "Please check if your library is compatible with Spark 
2.0", e)
+  } else {
+

[GitHub] spark pull request #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-25 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r84974785
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -462,13 +354,14 @@ case class DataSource(
 //  1. Only one output path can be specified on the write path;
 //  2. Output path must be a legal HDFS style file system path;
 //  3. It's OK that the output path doesn't exist yet;
-val caseInsensitiveOptions = new CaseInsensitiveMap(options)
-val outputPath = {
-  val path = new Path(caseInsensitiveOptions.getOrElse("path", {
-throw new IllegalArgumentException("'path' is not specified")
-  }))
+val allPaths = paths ++ new CaseInsensitiveMap(options).get("path")
+val outputPath = if (allPaths.length == 1) {
+  val path = new Path(allPaths.head)
   val fs = 
path.getFileSystem(sparkSession.sessionState.newHadoopConf())
   path.makeQualified(fs.getUri, fs.getWorkingDirectory)
+} else {
+  throw new IllegalArgumentException("At least one and only one 
path can be specified " +
--- End diff --

nit: "Expected exactly one path to be specified, but got..."


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-25 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r84975077
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -116,7 +116,7 @@ class InMemoryCatalog(
 fs.mkdirs(location)
   } catch {
 case e: IOException =>
-  throw new SparkException(s"Unable to create database 
${dbDefinition.name} as failed " +
+  throw new IOException(s"Unable to create database 
${dbDefinition.name} as failed " +
--- End diff --

Is it possible users depend on the conversion to SparkException? It seems 
safer to leave it as is.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-25 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r84975966
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -145,7 +145,7 @@ class InMemoryCatalog(
 fs.delete(location, true)
   } catch {
 case e: IOException =>
-  throw new SparkException(s"Unable to drop database 
${dbDefinition.name} as failed " +
+  throw new IOException(s"Unable to drop database 
${dbDefinition.name} as failed " +
--- End diff --

Not related to this PR, but just as a note I think it would be more 
grammatically correct to say "since deletion of ${..} failed" rather than "as 
we failed to delete ${..}".


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-24 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r84821321
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
@@ -91,7 +72,7 @@ case class CreateTableLikeCommand(
   CatalogTable(
 identifier = targetTable,
 tableType = CatalogTableType.MANAGED,
-storage = newStorage,
+storage = sourceTableDesc.storage.copy(locationUri = None),
--- End diff --

Add a comment as to why None?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-10-24 Thread ericl
Github user ericl commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r84818614
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -116,7 +116,7 @@ class InMemoryCatalog(
 fs.mkdirs(location)
   } catch {
 case e: IOException =>
-  throw new SparkException(s"Unable to create database 
${dbDefinition.name} as failed " +
+  throw new IOException(s"Unable to create database 
${dbDefinition.name} as failed " +
--- End diff --

Why change to IOException?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79328309
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -507,3 +400,117 @@ case class DataSource(
 }
   }
 }
+
+object DataSource {
+  /** A map to maintain backward compatibility in case we move data 
sources around. */
+  private val backwardCompatibilityMap: Map[String, String] = {
+val jdbc = classOf[JdbcRelationProvider].getCanonicalName
+val json = classOf[JsonFileFormat].getCanonicalName
+val parquet = classOf[ParquetFileFormat].getCanonicalName
+val csv = classOf[CSVFileFormat].getCanonicalName
+val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat"
+val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat"
+
+Map(
+  "org.apache.spark.sql.jdbc" -> jdbc,
+  "org.apache.spark.sql.jdbc.DefaultSource" -> jdbc,
+  "org.apache.spark.sql.execution.datasources.jdbc.DefaultSource" -> 
jdbc,
+  "org.apache.spark.sql.execution.datasources.jdbc" -> jdbc,
+  "org.apache.spark.sql.json" -> json,
+  "org.apache.spark.sql.json.DefaultSource" -> json,
+  "org.apache.spark.sql.execution.datasources.json" -> json,
+  "org.apache.spark.sql.execution.datasources.json.DefaultSource" -> 
json,
+  "org.apache.spark.sql.parquet" -> parquet,
+  "org.apache.spark.sql.parquet.DefaultSource" -> parquet,
+  "org.apache.spark.sql.execution.datasources.parquet" -> parquet,
+  "org.apache.spark.sql.execution.datasources.parquet.DefaultSource" 
-> parquet,
+  "org.apache.spark.sql.hive.orc.DefaultSource" -> orc,
+  "org.apache.spark.sql.hive.orc" -> orc,
+  "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm,
+  "org.apache.spark.ml.source.libsvm" -> libsvm,
+  "com.databricks.spark.csv" -> csv
+)
+  }
+
+  /**
+   * Class that were removed in Spark 2.0. Used to detect incompatibility 
libraries for Spark 2.0.
+   */
+  private val spark2RemovedClasses = Set(
+"org.apache.spark.sql.DataFrame",
+"org.apache.spark.sql.sources.HadoopFsRelationProvider",
+"org.apache.spark.Logging")
+
+  /** Given a provider name, look up the data source class definition. */
+  def lookupDataSource(provider0: String): Class[_] = {
+val provider = backwardCompatibilityMap.getOrElse(provider0, provider0)
+val provider2 = s"$provider.DefaultSource"
+val loader = Utils.getContextOrSparkClassLoader
+val serviceLoader = ServiceLoader.load(classOf[DataSourceRegister], 
loader)
+
+try {
+  
serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList 
match {
+// the provider format did not match any given registered aliases
+case Nil =>
+  try {
+
Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match {
+  case Success(dataSource) =>
+// Found the data source using fully qualified path
+dataSource
+  case Failure(error) =>
+if (provider.toLowerCase == "orc" ||
+  provider.startsWith("org.apache.spark.sql.hive.orc")) {
+  throw new AnalysisException(
+"The ORC data source must be used with Hive support 
enabled")
+} else if (provider.toLowerCase == "avro" ||
+  provider == "com.databricks.spark.avro") {
+  throw new AnalysisException(
+s"Failed to find data source: ${provider.toLowerCase}. 
Please find an Avro " +
+  "package at " +
+  
"https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects;)
+} else {
+  throw new ClassNotFoundException(
+s"Failed to find data source: $provider. Please find 
packages at " +
+  
"https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects;,
+error)
+}
+}
+  } catch {
+case e: NoClassDefFoundError => // This one won't be caught by 
Scala NonFatal
+  // NoClassDefFoundError's class name uses "/" rather than 
"." for packages
+  val className = e.getMessage.replaceAll("/", ".")
+  if (spark2RemovedClasses.contains(className)) {
+throw new ClassNotFoundException(s"$className was removed 
in Spark 2.0. " +
+  "Please check if your library is compatible with Spark 
2.0", e)
+  } else {
+

[GitHub] spark pull request #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79212283
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -409,16 +416,30 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+
+  val oldLocation = getLocationFromRawTable(oldTableDef)
+  // Only update the `locationUri` field if the location is really 
changed.
+  val newLocation = if (oldLocation == 
tableDefinition.storage.locationUri) {
--- End diff --

This is to keep the previous behaviour.

Previously, if we alter table fields (exclude location), we will keep the 
`locationUri` field and `path` option unchanged. If we alter table location, we 
will update both the `locationUri` field and `path` option, although it's 
dangerous to set `locationUri` as the path may be a file path.

Now, if we alter table fields (exclude location), we will keep the 
`locationUri` field unchanged, although the `locationUri` field may be 
different from the one in old raw table(the `oldTableDef` get from hive client 
directly) due to the hacks in `saveTableIntoHive`. If we alter table location, 
we will update the `locationUri` field even it's file path, which is the same 
behaviour as before.



---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79208945
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -409,16 +416,34 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+
+  val oldLocation = if (tableDefinition.tableType == EXTERNAL) {
+oldTableDef.properties.get(DATASOURCE_LOCATION)
+  } else {
+tableDefinition.storage.locationUri
+  }
+  // Only update the `locationUri` field if the location is really 
changed.
+  val newLocation = if (oldLocation == 
tableDefinition.storage.locationUri) {
--- End diff --

This is to keep the previous behaviour.

Previously, if we alter table fields (exclude location), we will keep the 
`locationUri` field and `path` option unchanged. If we alter table location, we 
will update both the `locationUri` field and `path` option, although it's 
dangerous to set `locationUri` as the path may be a file path.

Now, if we alter table fields (exclude location), we will keep the 
`locationUri` field unchanged, although the `locationUri` field may be 
different from the one in old raw table(the `oldTableDef` get from hive client 
directly) due to the hacks in `saveTableIntoHive`. If we alter table location, 
we will update the `locationUri` field even it's file path, which is the same 
behaviour as before.



---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79189646
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -409,16 +416,31 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 }
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
-  val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  val oldTableDef = client.getTable(db, 
withStatsProps.identifier.table)
+
+  val oldLocation = 
oldTableDef.properties.getOrElse(DATASOURCE_LOCATION,
+oldTableDef.storage.locationUri.get)
+  // Only update the `locationUri` field if the location is really 
changed.
+  val newLocation = if (oldLocation == 
tableDefinition.storage.locationUri.get) {
--- End diff --

This is to keep the previous behaviour.

Previously, if we alter table fields (exclude location), we will keep the 
`locationUri` field and `path` option unchanged. If we alter table location, we 
will update both the `locationUri` field and `path` option, although it's 
dangerous to set `locationUri` as the path may be a file path.

Now, if we alter table fields (exclude location), we will keep the 
`locationUri` field unchanged, although the `locationUri` field may be 
different from the one in old raw table(the `oldTableDef` get from hive client 
directly) due to the hacks in `saveTableIntoHive`. If we alter table location, 
we will update the `locationUri` field even it's file path, same as before.



---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79184660
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -665,15 +665,7 @@ case class AlterTableSetLocationCommand(
 catalog.alterPartitions(tableName, Seq(newPart))
   case None =>
 // No partition spec is specified, so we set the location for the 
table itself
-val newTable =
-  if (DDLUtils.isDatasourceTable(table)) {
-table.withNewStorage(
-  locationUri = Some(location),
-  properties = table.storage.properties ++ Map("path" -> 
location))
-  } else {
-table.withNewStorage(locationUri = Some(location))
-  }
-catalog.alterTable(newTable)
+catalog.alterTable(table.withNewStorage(locationUri = 
Some(location)))
--- End diff --

FYI we have a bug here, currently we allow users to `SET LOCATION` for 
managed data source table, however, in `SHOW CREATE TABLE`, we can't generate 
corrected SQL to create managed data source table whose location has been set, 
because data source with path is always treated as external table. We should 
either forbid `SET LOCATION` for managed data source table, or improve the 
CREATE TABLE syntax to support managed table with path.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79184683
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -665,15 +665,7 @@ case class AlterTableSetLocationCommand(
 catalog.alterPartitions(tableName, Seq(newPart))
   case None =>
 // No partition spec is specified, so we set the location for the 
table itself
-val newTable =
-  if (DDLUtils.isDatasourceTable(table)) {
-table.withNewStorage(
-  locationUri = Some(location),
-  properties = table.storage.properties ++ Map("path" -> 
location))
-  } else {
-table.withNewStorage(locationUri = Some(location))
-  }
-catalog.alterTable(newTable)
+catalog.alterTable(table.withNewStorage(locationUri = 
Some(location)))
--- End diff --

cc @yhuai @liancheng 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79183882
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -204,13 +194,21 @@ case class CreateDataSourceTableAsSelectCommand(
   case None => data
 }
 
+val tableWithPath = if (table.tableType == CatalogTableType.MANAGED) {
+  table.withNewStorage(
--- End diff --

see 
https://github.com/apache/spark/pull/15024/files#diff-a4847709cb46baee83fc4d3e8cc5c998R200


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79120516
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -410,15 +417,22 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
   val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  // get the data source properties from old table definition, and add 
the new location entry.
+  val dataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) ++
+tableDefinition.storage.locationUri.map { location =>
+  DATASOURCE_LOCATION -> location
+}
   val newDef = withStatsProps.copy(
+// TODO: we may break the hive-compatibility format for location 
URI here, we should follow
+// `createTable` and try to alter the table with `locationUri` 
set, if it's failed, then set
--- End diff --

if this is true, we may have a bug when create table with hive compatible 
format, i.e. we may create a managed table with a file path location, and 
successfully save it in hive metastore. Then we could get an exception when 
reading/writing this table.

cc @yhuai do we really wanna support file path location for data source 
tables?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79119480
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -410,15 +417,22 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
   val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  // get the data source properties from old table definition, and add 
the new location entry.
+  val dataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) ++
+tableDefinition.storage.locationUri.map { location =>
+  DATASOURCE_LOCATION -> location
+}
   val newDef = withStatsProps.copy(
+// TODO: we may break the hive-compatibility format for location 
URI here, we should follow
--- End diff --

cc @yhuai, shall we support `SET LOCATION` with file path?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79119231
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -665,15 +665,7 @@ case class AlterTableSetLocationCommand(
 catalog.alterPartitions(tableName, Seq(newPart))
   case None =>
 // No partition spec is specified, so we set the location for the 
table itself
-val newTable =
-  if (DDLUtils.isDatasourceTable(table)) {
-table.withNewStorage(
-  locationUri = Some(location),
-  properties = table.storage.properties ++ Map("path" -> 
location))
-  } else {
-table.withNewStorage(locationUri = Some(location))
-  }
-catalog.alterTable(newTable)
+catalog.alterTable(table.withNewStorage(locationUri = 
Some(location)))
--- End diff --

We supported it before, I'd like to not break it in this PR.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79119152
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -204,13 +194,21 @@ case class CreateDataSourceTableAsSelectCommand(
   case None => data
 }
 
+val tableWithPath = if (table.tableType == CatalogTableType.MANAGED) {
+  table.withNewStorage(
+locationUri = 
Some(sessionState.catalog.defaultTablePath(table.identifier)))
+} else {
+  table
+}
+
 // Create the relation based on the data of df.
 val dataSource = DataSource(
   sparkSession,
   className = provider,
-  partitionColumns = table.partitionColumnNames,
-  bucketSpec = table.bucketSpec,
-  options = optionsWithPath)
+  partitionColumns = tableWithPath.partitionColumnNames,
+  bucketSpec = tableWithPath.bucketSpec,
+  paths = tableWithPath.storage.locationUri.toSeq,
--- End diff --

I don't think this pattern is verbose and need a helper methood.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79119043
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -204,13 +194,21 @@ case class CreateDataSourceTableAsSelectCommand(
   case None => data
 }
 
+val tableWithPath = if (table.tableType == CatalogTableType.MANAGED) {
+  table.withNewStorage(
+locationUri = 
Some(sessionState.catalog.defaultTablePath(table.identifier)))
--- End diff --

it only appears here now


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79118965
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -195,18 +195,31 @@ class InMemoryCatalog(
 throw new TableAlreadyExistsException(db = db, table = table)
   }
 } else {
-  if (tableDefinition.tableType == CatalogTableType.MANAGED) {
-val dir = new Path(catalog(db).db.locationUri, table)
-try {
-  val fs = dir.getFileSystem(hadoopConfig)
-  fs.mkdirs(dir)
-} catch {
-  case e: IOException =>
-throw new SparkException(s"Unable to create table $table as 
failed " +
-  s"to create its directory $dir", e)
+  val tableWithLocation = if (tableDefinition.tableType == 
CatalogTableType.MANAGED) {
+val defaultTableLocation = new Path(catalog(db).db.locationUri, 
table)
+// Ideally we can not create a managed table with location, but 
due to some limitations in
+// [[CreateDataSourceTableAsSelectCommand]], we have to create the 
table directory and
+// write out data before we create this table. We should handle 
this case and allow the
+// table location to be pre-created, as long as it's same with the 
default table location.
+if (tableDefinition.storage.locationUri.isDefined) {
+  val givenTableLocation = new 
Path(tableDefinition.storage.locationUri.get).toUri.toString
+  require(defaultTableLocation.toUri.toString == 
givenTableLocation)
--- End diff --

this is not user facing, if we hit this, we have a bug


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79116875
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -447,17 +461,10 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 } else {
   getProviderFromTableProperties(table).map { provider =>
 assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+// Data source table always put its location URI(if it has) in 
table properties, to work
+// around a hive metastore issue. We should read it back before 
return the table metadata.
+val storage = if (table.tableType == EXTERNAL) {
--- End diff --

what do you mean? `HiveExternalCatlaog.alterTable` always add this location 
property.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78495300
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -435,13 +435,13 @@ case class DataSource(
 //  1. Only one output path can be specified on the write path;
 //  2. Output path must be a legal HDFS style file system path;
 //  3. It's OK that the output path doesn't exist yet;
-val caseInsensitiveOptions = new CaseInsensitiveMap(options)
-val outputPath = {
-  val path = new Path(caseInsensitiveOptions.getOrElse("path", {
-throw new IllegalArgumentException("'path' is not specified")
-  }))
+val allPaths = paths ++ new CaseInsensitiveMap(options).get("path")
--- End diff --

In the read path, multiple paths are allowed. However, the write path does 
not allow it. I searched the code base for building `DataSource`. Unable to 
find any data source instance that is constructed using both `paths` and 
`options` as input parameters.

However, it could be zero because of a bug mentioned above.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78493500
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -435,13 +435,13 @@ case class DataSource(
 //  1. Only one output path can be specified on the write path;
 //  2. Output path must be a legal HDFS style file system path;
 //  3. It's OK that the output path doesn't exist yet;
-val caseInsensitiveOptions = new CaseInsensitiveMap(options)
-val outputPath = {
-  val path = new Path(caseInsensitiveOptions.getOrElse("path", {
-throw new IllegalArgumentException("'path' is not specified")
-  }))
+val allPaths = paths ++ new CaseInsensitiveMap(options).get("path")
--- End diff --

Is it possible that both `paths` and `options` contains the same path 
string?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78493170
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -204,13 +194,21 @@ case class CreateDataSourceTableAsSelectCommand(
   case None => data
 }
 
+val tableWithPath = if (table.tableType == CatalogTableType.MANAGED) {
+  table.withNewStorage(
+locationUri = 
Some(sessionState.catalog.defaultTablePath(table.identifier)))
+} else {
+  table
+}
+
 // Create the relation based on the data of df.
 val dataSource = DataSource(
   sparkSession,
   className = provider,
-  partitionColumns = table.partitionColumnNames,
-  bucketSpec = table.bucketSpec,
-  options = optionsWithPath)
+  partitionColumns = tableWithPath.partitionColumnNames,
+  bucketSpec = tableWithPath.bucketSpec,
+  paths = tableWithPath.storage.locationUri.toSeq,
--- End diff --

```
tableWithPath.storage.locationUri
```
This patterns also appears multiple times.

Maybe create a helper method in class CatalogTable called location?
```
def location: Option[String] = this.storage.locationUri
``` 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78493048
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -204,13 +194,21 @@ case class CreateDataSourceTableAsSelectCommand(
   case None => data
 }
 
+val tableWithPath = if (table.tableType == CatalogTableType.MANAGED) {
+  table.withNewStorage(
+locationUri = 
Some(sessionState.catalog.defaultTablePath(table.identifier)))
--- End diff --

This pattern:
```
 val tableWithPath = if (table.tableType == CatalogTableType.MANAGED) {
   table.withNewStorage(
 locationUri = 
Some(sessionState.catalog.defaultTablePath(table.identifier)))
 ```
appears twice, may be we should create a util method in class CatalogTable? 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78487932
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -665,15 +665,7 @@ case class AlterTableSetLocationCommand(
 catalog.alterPartitions(tableName, Seq(newPart))
   case None =>
 // No partition spec is specified, so we set the location for the 
table itself
-val newTable =
-  if (DDLUtils.isDatasourceTable(table)) {
-table.withNewStorage(
-  locationUri = Some(location),
-  properties = table.storage.properties ++ Map("path" -> 
location))
-  } else {
-table.withNewStorage(locationUri = Some(location))
-  }
-catalog.alterTable(newTable)
+catalog.alterTable(table.withNewStorage(locationUri = 
Some(location)))
--- End diff --

For example, renaming a Hive managed table, we also change the location of 
this table. Thus, I think we might need to keep 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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78484539
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -665,15 +665,7 @@ case class AlterTableSetLocationCommand(
 catalog.alterPartitions(tableName, Seq(newPart))
   case None =>
 // No partition spec is specified, so we set the location for the 
table itself
-val newTable =
-  if (DDLUtils.isDatasourceTable(table)) {
-table.withNewStorage(
-  locationUri = Some(location),
-  properties = table.storage.properties ++ Map("path" -> 
location))
-  } else {
-table.withNewStorage(locationUri = Some(location))
-  }
-catalog.alterTable(newTable)
+catalog.alterTable(table.withNewStorage(locationUri = 
Some(location)))
--- End diff --

It is strange to allow user to change location of a managed table. Maybe we 
should ban this usage? 

Probably not in the scope of this PR.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78475748
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -253,6 +266,7 @@ class InMemoryCatalog(
   throw new SparkException(s"Unable to rename table $oldName to 
$newName as failed " +
 s"to rename its directory $oldDir", e)
   }
+  oldDesc.table = oldDesc.table.withNewStorage(locationUri = 
Some(newDir.toUri.toString))
--- End diff --

For determining the default location, we can do it based on your way. 
However, we should not disallow users to change it. `ALTER TABLE SET LOCATION` 
still can change the location of the managed table


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78475247
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -410,15 +417,22 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
   val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  // get the data source properties from old table definition, and add 
the new location entry.
+  val dataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) ++
+tableDefinition.storage.locationUri.map { location =>
+  DATASOURCE_LOCATION -> location
+}
   val newDef = withStatsProps.copy(
+// TODO: we may break the hive-compatibility format for location 
URI here, we should follow
+// `createTable` and try to alter the table with `locationUri` 
set, if it's failed, then set
--- End diff --

However, Hive normally does not return exceptions immediately. After 
incorrectly altering the table, we could get an exception when reading/writing 
this table. For example, we set the `locationUri` to `None` for a Hive managed 
table. We will hit an error when inserting the data into this table. When we 
drop the Hive-managed table, Hive should delete the directory pointed by 
`locationUri`. Then, if it is `None`, the directory will not be dropped.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78474896
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -410,15 +417,22 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
   val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  // get the data source properties from old table definition, and add 
the new location entry.
+  val dataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) ++
+tableDefinition.storage.locationUri.map { location =>
+  DATASOURCE_LOCATION -> location
+}
   val newDef = withStatsProps.copy(
+// TODO: we may break the hive-compatibility format for location 
URI here, we should follow
+// `createTable` and try to alter the table with `locationUri` 
set, if it's failed, then set
--- End diff --

I think this is based on the [codes in 
`createTable`](https://github.com/apache/spark/blob/d6eede9a36766e2d2294951b054d7557008a5662/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala#L313-L323)


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78474538
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -253,6 +266,7 @@ class InMemoryCatalog(
   throw new SparkException(s"Unable to rename table $oldName to 
$newName as failed " +
 s"to rename its directory $oldDir", e)
   }
+  oldDesc.table = oldDesc.table.withNewStorage(locationUri = 
Some(newDir.toUri.toString))
--- End diff --

We can hide the all hive specific details in HiveExternalCatalog? I see 
there is many code place where we try to add a locationUri for managed table. 
It doesn't seems necessary if the locationUri can be deduced from table name. 

We can always do some conversion in HiveExternalCatalog to change 
CatalogTable to hive understandable catalog.
 
```
private def saveTableIntoHive(tableDefinition: CatalogTable, 
ignoreIfExists: Boolean): Unit = {
```


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78473168
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -410,15 +417,22 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 
 if (DDLUtils.isDatasourceTable(withStatsProps)) {
   val oldDef = client.getTable(db, withStatsProps.identifier.table)
-  // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from 
the old table definition,
-  // to retain the spark specific format if it is. Also add old data 
source properties to table
-  // properties, to retain the data source table format.
-  val oldDataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX))
+  // get the data source properties from old table definition, and add 
the new location entry.
+  val dataSourceProps = 
oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) ++
+tableDefinition.storage.locationUri.map { location =>
+  DATASOURCE_LOCATION -> location
+}
   val newDef = withStatsProps.copy(
+// TODO: we may break the hive-compatibility format for location 
URI here, we should follow
+// `createTable` and try to alter the table with `locationUri` 
set, if it's failed, then set
--- End diff --

This comment is not very clear to me. What you mean by saying "follow 
`createTable` and try to alter the table with `locationUri` set"?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78472441
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala ---
@@ -262,11 +262,13 @@ class CatalogImpl(sparkSession: SparkSession) extends 
Catalog {
   throw new AnalysisException("Cannot create hive serde table with 
createExternalTable API.")
 }
 
+val location = new CaseInsensitiveMap(options).get("path")
+val storageProps = options.filterNot { case (k, _) => k.toLowerCase == 
"path" }
--- End diff --

I saw this two line duplicated in at least three places. Maybe we should 
create a util to extract the path from options?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78472391
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -665,15 +665,7 @@ case class AlterTableSetLocationCommand(
 catalog.alterPartitions(tableName, Seq(newPart))
   case None =>
 // No partition spec is specified, so we set the location for the 
table itself
-val newTable =
-  if (DDLUtils.isDatasourceTable(table)) {
-table.withNewStorage(
-  locationUri = Some(location),
-  properties = table.storage.properties ++ Map("path" -> 
location))
-  } else {
-table.withNewStorage(locationUri = Some(location))
-  }
-catalog.alterTable(newTable)
+catalog.alterTable(table.withNewStorage(locationUri = 
Some(location)))
--- End diff --

It is legal to change the location for Hive managed tables. Why issuing an 
exception here?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78472272
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -435,13 +435,13 @@ case class DataSource(
 //  1. Only one output path can be specified on the write path;
 //  2. Output path must be a legal HDFS style file system path;
 //  3. It's OK that the output path doesn't exist yet;
-val caseInsensitiveOptions = new CaseInsensitiveMap(options)
-val outputPath = {
-  val path = new Path(caseInsensitiveOptions.getOrElse("path", {
-throw new IllegalArgumentException("'path' is not specified")
-  }))
+val allPaths = paths ++ new CaseInsensitiveMap(options).get("path")
+val outputPath = if (allPaths.length == 1) {
+  val path = new Path(allPaths.head)
   val fs = 
path.getFileSystem(sparkSession.sessionState.newHadoopConf())
   path.makeQualified(fs.getUri, fs.getWorkingDirectory)
+} else {
+  throw new IllegalArgumentException("Only one path can be 
specified on the write path")
--- End diff --

Maybe the error message should be more explicit? Like listing what is 
current paths, and what is the path in option? And how to fix it?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78472224
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -435,13 +435,13 @@ case class DataSource(
 //  1. Only one output path can be specified on the write path;
 //  2. Output path must be a legal HDFS style file system path;
 //  3. It's OK that the output path doesn't exist yet;
-val caseInsensitiveOptions = new CaseInsensitiveMap(options)
-val outputPath = {
-  val path = new Path(caseInsensitiveOptions.getOrElse("path", {
-throw new IllegalArgumentException("'path' is not specified")
-  }))
+val allPaths = paths ++ new CaseInsensitiveMap(options).get("path")
+val outputPath = if (allPaths.length == 1) {
+  val path = new Path(allPaths.head)
   val fs = 
path.getFileSystem(sparkSession.sessionState.newHadoopConf())
   path.makeQualified(fs.getUri, fs.getWorkingDirectory)
+} else {
+  throw new IllegalArgumentException("Only one path can be 
specified on the write path")
--- End diff --

Is IllegalArgumentException a proper message type? It is a RuntimeException 
used when calling a method with reflection but passing different argument type.

```
/**
 * Thrown to indicate that a method has been passed an illegal or
 * inappropriate argument.
 *
 * @author  unascribed
 * @since   JDK1.0
 */
``` 




---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78471429
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala ---
@@ -665,15 +665,7 @@ case class AlterTableSetLocationCommand(
 catalog.alterPartitions(tableName, Seq(newPart))
   case None =>
 // No partition spec is specified, so we set the location for the 
table itself
-val newTable =
-  if (DDLUtils.isDatasourceTable(table)) {
-table.withNewStorage(
-  locationUri = Some(location),
-  properties = table.storage.properties ++ Map("path" -> 
location))
-  } else {
-table.withNewStorage(locationUri = Some(location))
-  }
-catalog.alterTable(newTable)
+catalog.alterTable(table.withNewStorage(locationUri = 
Some(location)))
--- End diff --

Should we throws an exception if it is a managed table?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78471404
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -253,6 +266,7 @@ class InMemoryCatalog(
   throw new SparkException(s"Unable to rename table $oldName to 
$newName as failed " +
 s"to rename its directory $oldDir", e)
   }
+  oldDesc.table = oldDesc.table.withNewStorage(locationUri = 
Some(newDir.toUri.toString))
--- End diff --

In Hive, users are allowed to create a Hive managed table with the 
user-specified location. In Spark SQL, we do not allow users to do it. If users 
specify the location, we always convert the type to EXTERNAL.

When we creating managed table without a location, Hive will set it for us. 
Before this PR, sometimes we check `path` and sometimes we check `locationUri`. 
It could hide bugs we do not realize, especially for 
`CreateDataSourceTableAsSelectCommand`. We assume our generated `path` is 
always identical to the `locationUri` that is populated by Hive. Thus, we 
should explicitly set `locationUri` for Hive managed table using our generated 
`path`.

Based on my understanding, here is to ensure `InMemoryCatalog` and 
`HiveExternalCatalog` behave the same. 




---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78471303
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -204,13 +194,21 @@ case class CreateDataSourceTableAsSelectCommand(
   case None => data
 }
 
+val tableWithPath = if (table.tableType == CatalogTableType.MANAGED) {
+  table.withNewStorage(
--- End diff --

I think we should not set location Uri for managed table. As the location 
Uri can be deduced by database path and table name.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78471240
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -204,13 +194,21 @@ case class CreateDataSourceTableAsSelectCommand(
   case None => data
 }
 
+val tableWithPath = if (table.tableType == CatalogTableType.MANAGED) {
+  table.withNewStorage(
+locationUri = 
Some(sessionState.catalog.defaultTablePath(table.identifier)))
--- End diff --

I think we should add a method in CatalogTable to return the location Uri 
by combining database path + table name.



---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78471137
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala
 ---
@@ -154,13 +149,8 @@ case class CreateDataSourceTableAsSelectCommand(
   return Seq.empty[Row]
 case SaveMode.Append =>
   // Check if the specified data source match the data source of 
the existing table.
-  val dataSource = DataSource(
-sparkSession = sparkSession,
-userSpecifiedSchema = Some(query.schema.asNullable),
-partitionColumns = table.partitionColumnNames,
-bucketSpec = table.bucketSpec,
-className = provider,
-options = optionsWithPath)
+  val previousProvider =
--- End diff --

Maybe rename as existingFileFormat or existingProvider?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78469005
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -253,6 +266,7 @@ class InMemoryCatalog(
   throw new SparkException(s"Unable to rename table $oldName to 
$newName as failed " +
 s"to rename its directory $oldDir", e)
   }
+  oldDesc.table = oldDesc.table.withNewStorage(locationUri = 
Some(newDir.toUri.toString))
--- End diff --

Maybe for managed table, we should never set field locationUri in.




---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78468230
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -195,18 +195,31 @@ class InMemoryCatalog(
 throw new TableAlreadyExistsException(db = db, table = table)
   }
 } else {
-  if (tableDefinition.tableType == CatalogTableType.MANAGED) {
-val dir = new Path(catalog(db).db.locationUri, table)
-try {
-  val fs = dir.getFileSystem(hadoopConfig)
-  fs.mkdirs(dir)
-} catch {
-  case e: IOException =>
-throw new SparkException(s"Unable to create table $table as 
failed " +
-  s"to create its directory $dir", e)
+  val tableWithLocation = if (tableDefinition.tableType == 
CatalogTableType.MANAGED) {
+val defaultTableLocation = new Path(catalog(db).db.locationUri, 
table)
+// Ideally we can not create a managed table with location, but 
due to some limitations in
+// [[CreateDataSourceTableAsSelectCommand]], we have to create the 
table directory and
+// write out data before we create this table. We should handle 
this case and allow the
+// table location to be pre-created, as long as it's same with the 
default table location.
+if (tableDefinition.storage.locationUri.isDefined) {
+  val givenTableLocation = new 
Path(tableDefinition.storage.locationUri.get).toUri.toString
+  require(defaultTableLocation.toUri.toString == 
givenTableLocation)
+  tableDefinition
+} else {
+  try {
+val fs = defaultTableLocation.getFileSystem(hadoopConfig)
+fs.mkdirs(defaultTableLocation)
+  } catch {
+case e: IOException =>
+  throw new SparkException(s"Unable to create table $table as 
failed " +
--- End diff --

Maybe we should use IOException type instead of SparkException?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78468181
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -195,18 +195,31 @@ class InMemoryCatalog(
 throw new TableAlreadyExistsException(db = db, table = table)
   }
 } else {
-  if (tableDefinition.tableType == CatalogTableType.MANAGED) {
-val dir = new Path(catalog(db).db.locationUri, table)
-try {
-  val fs = dir.getFileSystem(hadoopConfig)
-  fs.mkdirs(dir)
-} catch {
-  case e: IOException =>
-throw new SparkException(s"Unable to create table $table as 
failed " +
-  s"to create its directory $dir", e)
+  val tableWithLocation = if (tableDefinition.tableType == 
CatalogTableType.MANAGED) {
+val defaultTableLocation = new Path(catalog(db).db.locationUri, 
table)
+// Ideally we can not create a managed table with location, but 
due to some limitations in
+// [[CreateDataSourceTableAsSelectCommand]], we have to create the 
table directory and
+// write out data before we create this table. We should handle 
this case and allow the
+// table location to be pre-created, as long as it's same with the 
default table location.
+if (tableDefinition.storage.locationUri.isDefined) {
+  val givenTableLocation = new 
Path(tableDefinition.storage.locationUri.get).toUri.toString
+  require(defaultTableLocation.toUri.toString == 
givenTableLocation)
--- End diff --

`require(defaultTableLocation.toUri.toString == givenTableLocation)` 
doesn't give a clear user-facing message. Should we replace it with an explicit 
Exception?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread clockfly
Github user clockfly commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78468093
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
 ---
@@ -195,18 +195,31 @@ class InMemoryCatalog(
 throw new TableAlreadyExistsException(db = db, table = table)
   }
 } else {
-  if (tableDefinition.tableType == CatalogTableType.MANAGED) {
-val dir = new Path(catalog(db).db.locationUri, table)
-try {
-  val fs = dir.getFileSystem(hadoopConfig)
-  fs.mkdirs(dir)
-} catch {
-  case e: IOException =>
-throw new SparkException(s"Unable to create table $table as 
failed " +
-  s"to create its directory $dir", e)
+  val tableWithLocation = if (tableDefinition.tableType == 
CatalogTableType.MANAGED) {
+val defaultTableLocation = new Path(catalog(db).db.locationUri, 
table)
+// Ideally we can not create a managed table with location, but 
due to some limitations in
+// [[CreateDataSourceTableAsSelectCommand]], we have to create the 
table directory and
+// write out data before we create this table. We should handle 
this case and allow the
+// table location to be pre-created, as long as it's same with the 
default table location.
+if (tableDefinition.storage.locationUri.isDefined) {
+  val givenTableLocation = new 
Path(tableDefinition.storage.locationUri.get).toUri.toString
+  require(defaultTableLocation.toUri.toString == 
givenTableLocation)
--- End diff --

Can we unify the if and else branch? Should we also check whether 
defaultTableLocation directory path is created if 
tableDefinition.storage.locationUri is set?


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-12 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r78433410
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -447,17 +461,10 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 } else {
   getProviderFromTableProperties(table).map { provider =>
 assert(provider != "hive", "Hive serde table should not save 
provider in table properties.")
-// SPARK-15269: Persisted data source tables always store the 
location URI as a storage
-// property named "path" instead of standard Hive `dataLocation`, 
because Hive only
-// allows directory paths as location URIs while Spark SQL data 
source tables also
-// allows file paths. So the standard Hive `dataLocation` is 
meaningless for Spark SQL
-// data source tables.
-// Spark SQL may also save external data source in Hive compatible 
format when
-// possible, so that these tables can be directly accessed by 
Hive. For these tables,
-// `dataLocation` is still necessary. Here we also check for input 
format because only
-// these Hive compatible tables set this field.
-val storage = if (table.tableType == EXTERNAL && 
table.storage.inputFormat.isEmpty) {
-  table.storage.copy(locationUri = None)
+// Data source table always put its location URI(if it has) in 
table properties, to work
+// around a hive metastore issue. We should read it back before 
return the table metadata.
+val storage = if (table.tableType == EXTERNAL) {
--- End diff --

For managed tables, do we need to set  
`table.properties.get(DATASOURCE_LOCATION)` to `locationUri`? Previously, Hive 
does it for us. Now, we explicitly remove it at multiple places. For example, 
`alterTable` removes it for both external tables and managed tables.


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-09 Thread cloud-fan
GitHub user cloud-fan opened a pull request:

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

[SPARK-17470][SQL] unify path for data source table and locationUri for 
hive serde table

## What changes were proposed in this pull request?

Due to a limitation of hive metastore(table location must be directory 
path, not file path), we always store `path` for data source table in storage 
properties, instead of the `locationUri` field. However, we should not expose 
this difference to `CatalogTable` level, but just treat it as a hack in 
`HiveExternalCatalog`, like we store table schema of data source table in table 
properties.

This PR unifies `path` and `locationUri` outside of `HiveExternalCatalog`, 
both data source table and hive serde table should use the `locationUri` field.

This PR also unifies the way we handle default table location for managed 
table. Previously, the default table location of hive serde managed table is 
set by external catalog, but the one of data source table is set by command. 
After this PR, we follow the hive way and the default table location is always 
set by external catalog.

## How was this patch tested?

existing tests.



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

$ git pull https://github.com/cloud-fan/spark path

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

https://github.com/apache/spark/pull/15024.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 #15024


commit 978c94cf25c4c6c19a693d7028573c43f160ee80
Author: Wenchen Fan 
Date:   2016-09-09T13:08:38Z

unify path for data source table and locationUri for hive serde table




---
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