[GitHub] spark pull request #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-28 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

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

https://github.com/apache/spark/pull/17015#discussion_r103414883
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala ---
@@ -1,179 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import java.io.IOException
-
-import com.google.common.base.Objects
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.hive.common.StatsSetupConst
-import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable}
-import org.apache.hadoop.hive.ql.plan.TableDesc
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.CatalystConf
-import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
-import org.apache.spark.sql.catalyst.catalog._
-import org.apache.spark.sql.catalyst.expressions.{AttributeMap, 
AttributeReference, Expression}
-import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, 
Statistics}
-import org.apache.spark.sql.execution.FileRelation
-import org.apache.spark.sql.hive.client.HiveClientImpl
-import org.apache.spark.sql.types.StructField
-
-
-private[hive] case class MetastoreRelation(
-databaseName: String,
-tableName: String)
-(val catalogTable: CatalogTable,
- @transient private val sparkSession: SparkSession)
-  extends LeafNode with MultiInstanceRelation with FileRelation with 
CatalogRelation {
-
-  override def equals(other: Any): Boolean = other match {
-case relation: MetastoreRelation =>
-  databaseName == relation.databaseName &&
-tableName == relation.tableName &&
-output == relation.output
-case _ => false
-  }
-
-  override def hashCode(): Int = {
-Objects.hashCode(databaseName, tableName, output)
-  }
-
-  override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: 
sparkSession :: Nil
-
-  @transient val hiveQlTable: HiveTable = 
HiveClientImpl.toHiveTable(catalogTable)
-
-  @transient override def computeStats(conf: CatalystConf): Statistics = {
-catalogTable.stats.map(_.toPlanStats(output)).getOrElse(Statistics(
-  sizeInBytes = {
-val totalSize = 
hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE)
-val rawDataSize = 
hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE)
-// TODO: check if this estimate is valid for tables after 
partition pruning.
-// NOTE: getting `totalSize` directly from params is kind of 
hacky, but this should be
-// relatively cheap if parameters for the table are populated into 
the metastore.
-// Besides `totalSize`, there are also `numFiles`, `numRows`, 
`rawDataSize` keys
-// (see StatsSetupConst in Hive) that we can look at in the future.
-BigInt(
-  // When table is external,`totalSize` is always zero, which will 
influence join strategy
-  // so when `totalSize` is zero, use `rawDataSize` instead
-  // when `rawDataSize` is also zero, use 
`HiveExternalCatalog.STATISTICS_TOTAL_SIZE`,
-  // which is generated by analyze command.
-  if (totalSize != null && totalSize.toLong > 0L) {
-totalSize.toLong
-  } else if (rawDataSize != null && rawDataSize.toLong > 0) {
-rawDataSize.toLong
-  } else if 
(sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled) {
-try {
-  val hadoopConf = sparkSession.sessionState.newHadoopConf()
-  val fs: FileSystem = 
hiveQlTable.getPath.getFileSystem(hadoopConf)
-  fs.getContentSummary(hiveQlTable.getPath).getLength
-} catch {
-  case e: IOException =>
-logWarning("Failed to get table size from hdfs.", e)
-sparkSession.sessionState.conf.defaultSizeInBytes
-}
-  

[GitHub] spark pull request #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r103393810
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala ---
@@ -1,179 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import java.io.IOException
-
-import com.google.common.base.Objects
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.hive.common.StatsSetupConst
-import org.apache.hadoop.hive.ql.metadata.{Partition, Table => HiveTable}
-import org.apache.hadoop.hive.ql.plan.TableDesc
-
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.CatalystConf
-import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
-import org.apache.spark.sql.catalyst.catalog._
-import org.apache.spark.sql.catalyst.expressions.{AttributeMap, 
AttributeReference, Expression}
-import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, 
Statistics}
-import org.apache.spark.sql.execution.FileRelation
-import org.apache.spark.sql.hive.client.HiveClientImpl
-import org.apache.spark.sql.types.StructField
-
-
-private[hive] case class MetastoreRelation(
-databaseName: String,
-tableName: String)
-(val catalogTable: CatalogTable,
- @transient private val sparkSession: SparkSession)
-  extends LeafNode with MultiInstanceRelation with FileRelation with 
CatalogRelation {
-
-  override def equals(other: Any): Boolean = other match {
-case relation: MetastoreRelation =>
-  databaseName == relation.databaseName &&
-tableName == relation.tableName &&
-output == relation.output
-case _ => false
-  }
-
-  override def hashCode(): Int = {
-Objects.hashCode(databaseName, tableName, output)
-  }
-
-  override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: 
sparkSession :: Nil
-
-  @transient val hiveQlTable: HiveTable = 
HiveClientImpl.toHiveTable(catalogTable)
-
-  @transient override def computeStats(conf: CatalystConf): Statistics = {
-catalogTable.stats.map(_.toPlanStats(output)).getOrElse(Statistics(
-  sizeInBytes = {
-val totalSize = 
hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE)
-val rawDataSize = 
hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE)
-// TODO: check if this estimate is valid for tables after 
partition pruning.
-// NOTE: getting `totalSize` directly from params is kind of 
hacky, but this should be
-// relatively cheap if parameters for the table are populated into 
the metastore.
-// Besides `totalSize`, there are also `numFiles`, `numRows`, 
`rawDataSize` keys
-// (see StatsSetupConst in Hive) that we can look at in the future.
-BigInt(
-  // When table is external,`totalSize` is always zero, which will 
influence join strategy
-  // so when `totalSize` is zero, use `rawDataSize` instead
-  // when `rawDataSize` is also zero, use 
`HiveExternalCatalog.STATISTICS_TOTAL_SIZE`,
-  // which is generated by analyze command.
-  if (totalSize != null && totalSize.toLong > 0L) {
-totalSize.toLong
-  } else if (rawDataSize != null && rawDataSize.toLong > 0) {
-rawDataSize.toLong
-  } else if 
(sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled) {
-try {
-  val hadoopConf = sparkSession.sessionState.newHadoopConf()
-  val fs: FileSystem = 
hiveQlTable.getPath.getFileSystem(hadoopConf)
-  fs.getContentSummary(hiveQlTable.getPath).getLength
-} catch {
-  case e: IOException =>
-logWarning("Failed to get table size from hdfs.", e)
-sparkSession.sessionState.conf.defaultSizeInBytes
-}
- 

[GitHub] spark pull request #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r103391249
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
 ---
@@ -138,27 +153,35 @@ case class HiveTableScanExec(
 }
   }
 
+  // exposed for tests
+  @transient lazy val rawPartitions = {
+val prunedPartitions = if 
(sparkSession.sessionState.conf.metastorePartitionPruning) {
--- End diff --

Checked the history. It sounds like @liancheng can answer whether this is 
still needed or not. : )

https://github.com/apache/spark/pull/7421#issuecomment-122527391 


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r103387597
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
@@ -40,38 +38,24 @@ case class AnalyzeColumnCommand(
 val sessionState = sparkSession.sessionState
 val db = 
tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase)
 val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db))
-val relation =
-  
EliminateSubqueryAliases(sparkSession.table(tableIdentWithDB).queryExecution.analyzed)
-
-// Compute total size
-val (catalogTable: CatalogTable, sizeInBytes: Long) = relation match {
-  case catalogRel: CatalogRelation =>
-// This is a Hive serde format table
-(catalogRel.catalogTable,
-  AnalyzeTableCommand.calculateTotalSize(sessionState, 
catalogRel.catalogTable))
-
-  case logicalRel: LogicalRelation if 
logicalRel.catalogTable.isDefined =>
-// This is a data source format table
-(logicalRel.catalogTable.get,
-  AnalyzeTableCommand.calculateTotalSize(sessionState, 
logicalRel.catalogTable.get))
-
-  case otherRelation =>
-throw new AnalysisException("ANALYZE TABLE is not supported for " +
-  s"${otherRelation.nodeName}.")
+val tableMeta = sessionState.catalog.getTableMetadata(tableIdentWithDB)
+if (tableMeta.tableType == CatalogTableType.VIEW) {
+  throw new AnalysisException("ANALYZE TABLE is not supported on 
views.")
 }
+val sizeInBytes = AnalyzeTableCommand.calculateTotalSize(sessionState, 
tableMeta)
 
 // Compute stats for each column
 val (rowCount, newColStats) =
-  AnalyzeColumnCommand.computeColumnStats(sparkSession, 
tableIdent.table, relation, columnNames)
+  AnalyzeColumnCommand.computeColumnStats(sparkSession, 
tableIdentWithDB, columnNames)
--- End diff --

`object AnalyzeColumnCommand` is not needed. We can move 
`computeColumnStats ` into the `case class AnalyzeColumnCommand`


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r103387529
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
@@ -90,10 +74,10 @@ object AnalyzeColumnCommand extends Logging {
*/
   def computeColumnStats(
--- End diff --

Now, this is not being used for testing. We can mark it as private. 


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r103383919
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala ---
@@ -91,18 +97,58 @@ class ResolveHiveSerdeTable(session: SparkSession) 
extends Rule[LogicalPlan] {
 
   // Infers the schema, if empty, because the schema could be 
determined by Hive
   // serde.
-  val catalogTable = if (query.isEmpty) {
-val withSchema = HiveUtils.inferSchema(withStorage)
-if (withSchema.schema.length <= 0) {
+  val withSchema = if (query.isEmpty) {
+val inferred = HiveUtils.inferSchema(withStorage)
+if (inferred.schema.length <= 0) {
   throw new AnalysisException("Unable to infer the schema. " +
-s"The schema specification is required to create the table 
${withSchema.identifier}.")
+s"The schema specification is required to create the table 
${inferred.identifier}.")
 }
-withSchema
+inferred
   } else {
 withStorage
   }
 
-  c.copy(tableDesc = catalogTable)
+  c.copy(tableDesc = withSchema)
+  }
+}
+
+class DetermineTableStats(session: SparkSession) extends Rule[LogicalPlan] 
{
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case relation: CatalogRelation
+if DDLUtils.isHiveTable(relation.tableMeta) && 
relation.tableMeta.stats.isEmpty =>
+  val table = relation.tableMeta
+  // TODO: check if this estimate is valid for tables after partition 
pruning.
+  // NOTE: getting `totalSize` directly from params is kind of hacky, 
but this should be
+  // relatively cheap if parameters for the table are populated into 
the metastore.
+  // Besides `totalSize`, there are also `numFiles`, `numRows`, 
`rawDataSize` keys
+  // (see StatsSetupConst in Hive) that we can look at in the future.
+  // When table is external,`totalSize` is always zero, which will 
influence join strategy
+  // so when `totalSize` is zero, use `rawDataSize` instead
+  // when `rawDataSize` is also zero, use 
`HiveExternalCatalog.STATISTICS_TOTAL_SIZE`,
--- End diff --

This is out of dated, I think


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-27 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r103383279
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -349,36 +350,41 @@ object CatalogTypes {
 
 
 /**
- * An interface that is implemented by logical plans to return the 
underlying catalog table.
- * If we can in the future consolidate SimpleCatalogRelation and 
MetastoreRelation, we should
- * probably remove this interface.
+ * A [[LogicalPlan]] that represents a table.
  */
-trait CatalogRelation {
-  def catalogTable: CatalogTable
-  def output: Seq[Attribute]
-}
+case class CatalogRelation(
+tableMeta: CatalogTable,
+dataCols: Seq[Attribute],
+partitionCols: Seq[Attribute]) extends LeafNode with 
MultiInstanceRelation {
+  assert(tableMeta.identifier.database.isDefined)
+  assert(tableMeta.partitionSchema.sameType(partitionCols.toStructType))
+  assert(tableMeta.dataSchema.sameType(dataCols.toStructType))
+
+  // The partition column should always appear after data columns.
+  override def output: Seq[Attribute] = dataCols ++ partitionCols
+
+  def isPartitioned: Boolean = partitionCols.nonEmpty
+
+  override def equals(relation: Any): Boolean = relation match {
+case other: CatalogRelation => tableMeta == other.tableMeta && output 
== other.output
+case _ => false
+  }
 
+  override def hashCode(): Int = {
+Objects.hashCode(tableMeta.identifier, output)
+  }
 
-/**
- * A [[LogicalPlan]] that wraps [[CatalogTable]].
- *
- * Note that in the future we should consolidate this and 
HiveCatalogRelation.
- */
-case class SimpleCatalogRelation(
-metadata: CatalogTable)
-  extends LeafNode with CatalogRelation {
-
-  override def catalogTable: CatalogTable = metadata
-
-  override lazy val resolved: Boolean = false
-
-  override val output: Seq[Attribute] = {
-val (partCols, dataCols) = metadata.schema.toAttributes
-  // Since data can be dumped in randomly with no validation, 
everything is nullable.
-  
.map(_.withNullability(true).withQualifier(Some(metadata.identifier.table)))
-  .partition { a =>
-metadata.partitionColumnNames.contains(a.name)
-  }
-dataCols ++ partCols
+  /** Only compare table identifier. */
+  override lazy val cleanArgs: Seq[Any] = Seq(tableMeta.identifier)
+
+  override def computeStats(conf: CatalystConf): Statistics = {
+// For data source tables, we will create a `LogicalRelation` and 
won't call this method, for
+// hive serde tables, we will always generate a statistics.
+// TODO: unify the table stats generation.
+tableMeta.stats.map(_.toPlanStats(output)).get
--- End diff --

Yeah, the value should be always filled by `DetermineTableStats`, but maybe 
we still can issue an exception when it is `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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102825895
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala 
---
@@ -1,55 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.spark.sql.hive
-
-import org.apache.spark.sql.{QueryTest, Row}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, 
CatalogTable, CatalogTableType}
-import org.apache.spark.sql.hive.test.TestHiveSingleton
-import org.apache.spark.sql.test.SQLTestUtils
-import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
-
-class MetastoreRelationSuite extends QueryTest with SQLTestUtils with 
TestHiveSingleton {
-  test("makeCopy and toJSON should work") {
-val table = CatalogTable(
-  identifier = TableIdentifier("test", Some("db")),
-  tableType = CatalogTableType.VIEW,
-  storage = CatalogStorageFormat.empty,
-  schema = StructType(StructField("a", IntegerType, true) :: Nil))
-val relation = MetastoreRelation("db", "test")(table, null)
-
-// No exception should be thrown
-relation.makeCopy(Array("db", "test"))
-// No exception should be thrown
-relation.toJSON
-  }
-
-  test("SPARK-17409: Do Not Optimize Query in CTAS (Hive Serde Table) More 
Than Once") {
--- End diff --

Keep this test case?


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102825217
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 ---
@@ -74,7 +74,7 @@ import org.apache.spark.SparkException
  * @param ifNotExists If true, only write if the table or partition does 
not exist.
  */
 case class InsertIntoHiveTable(
-table: MetastoreRelation,
+table: CatalogRelation,
--- End diff --

Like the comment in the original `@param` , how about replacing it by 
`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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102822414
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
 ---
@@ -29,29 +30,31 @@ import 
org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.catalog.CatalogRelation
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution._
 import org.apache.spark.sql.execution.metric.SQLMetrics
 import org.apache.spark.sql.hive._
+import org.apache.spark.sql.hive.client.HiveClientImpl
 import org.apache.spark.sql.types.{BooleanType, DataType}
 import org.apache.spark.util.Utils
 
 /**
  * The Hive table scan operator.  Column and partition pruning are both 
handled.
  *
  * @param requestedAttributes Attributes to be fetched from the Hive table.
- * @param relation The Hive table be scanned.
+ * @param relation the Hive table be scanned.
--- End diff --

Keep it 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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102821580
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -100,52 +101,49 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   None
 }
   case _ =>
-logWarning(
-  
s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} " +
-s"should be stored as $expectedFileFormat. However, we are 
getting " +
-s"a ${relation.fileFormat} from the metastore cache. This 
cached " +
-s"entry will be invalidated.")
+logWarning(s"Table $tableIdentifier should be stored as 
$expectedFileFormat. " +
+  s"However, we are getting a ${relation.fileFormat} from the 
metastore cache. " +
+  "This cached entry will be invalidated.")
 tableRelationCache.invalidate(tableIdentifier)
 None
 }
   case other =>
-logWarning(
-  
s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} should be 
stored " +
-s"as $expectedFileFormat. However, we are getting a $other 
from the metastore cache. " +
-s"This cached entry will be invalidated.")
+logWarning(s"Table $tableIdentifier should be stored as 
$expectedFileFormat. " +
+  s"However, we are getting a $other from the metastore cache. " +
+  "This cached entry will be invalidated.")
 tableRelationCache.invalidate(tableIdentifier)
 None
 }
   }
 
   private def convertToLogicalRelation(
-  metastoreRelation: MetastoreRelation,
+  relation: CatalogRelation,
   options: Map[String, String],
-  defaultSource: FileFormat,
   fileFormatClass: Class[_ <: FileFormat],
   fileType: String): LogicalRelation = {
-val metastoreSchema = 
StructType.fromAttributes(metastoreRelation.output)
+val metastoreSchema = relation.tableMeta.schema
 val tableIdentifier =
-  QualifiedTableName(metastoreRelation.databaseName, 
metastoreRelation.tableName)
-val bucketSpec = None  // We don't support hive bucketed tables, only 
ones we write out.
+  QualifiedTableName(relation.tableMeta.database, 
relation.tableMeta.identifier.table)
 
 val lazyPruningEnabled = 
sparkSession.sqlContext.conf.manageFilesourcePartitions
-val result = if (metastoreRelation.hiveQlTable.isPartitioned) {
-  val partitionSchema = 
StructType.fromAttributes(metastoreRelation.partitionKeys)
-
+val tablePath = new Path(new URI(relation.tableMeta.location))
+val result = if (relation.isPartitioned) {
+  val partitionSchema = relation.tableMeta.partitionSchema
   val rootPaths: Seq[Path] = if (lazyPruningEnabled) {
-Seq(metastoreRelation.hiveQlTable.getDataLocation)
+Seq(tablePath)
   } else {
 // By convention (for example, see CatalogFileIndex), the 
definition of a
 // partitioned table's paths depends on whether that table has any 
actual partitions.
 // Partitioned tables without partitions use the location of the 
table's base path.
 // Partitioned tables with partitions use the locations of those 
partitions' data
 // locations,_omitting_ the table's base path.
-val paths = metastoreRelation.getHiveQlPartitions().map { p =>
-  new Path(p.getLocation)
+val paths = 
sparkSession.sharedState.externalCatalog.listPartitions(
+  tableIdentifier.database, tableIdentifier.name).map { p =>
+  new Path(new URI(p.storage.locationUri.get))
--- End diff --

indent?


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102821050
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -155,66 +153,58 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
 val cached = getCached(
   tableIdentifier,
   rootPaths,
-  metastoreRelation,
   metastoreSchema,
   fileFormatClass,
-  bucketSpec,
   Some(partitionSchema))
 
 val logicalRelation = cached.getOrElse {
-  val sizeInBytes =
-
metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
+  val sizeInBytes = 
relation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
   val fileIndex = {
-val index = new CatalogFileIndex(
-  sparkSession, metastoreRelation.catalogTable, sizeInBytes)
+val index = new CatalogFileIndex(sparkSession, 
relation.tableMeta, sizeInBytes)
 if (lazyPruningEnabled) {
   index
 } else {
   index.filterPartitions(Nil)  // materialize all the 
partitions in memory
 }
   }
-  val partitionSchemaColumnNames = 
partitionSchema.map(_.name.toLowerCase).toSet
-  val dataSchema =
-StructType(metastoreSchema
-  .filterNot(field => 
partitionSchemaColumnNames.contains(field.name.toLowerCase)))
+  val dataSchema = relation.tableMeta.dataSchema
--- End diff --

This is just being used in only one place. We can get rid of 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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102818575
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -155,66 +153,58 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
 val cached = getCached(
   tableIdentifier,
   rootPaths,
-  metastoreRelation,
   metastoreSchema,
   fileFormatClass,
-  bucketSpec,
   Some(partitionSchema))
 
 val logicalRelation = cached.getOrElse {
-  val sizeInBytes =
-
metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
+  val sizeInBytes = 
relation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
   val fileIndex = {
-val index = new CatalogFileIndex(
-  sparkSession, metastoreRelation.catalogTable, sizeInBytes)
+val index = new CatalogFileIndex(sparkSession, 
relation.tableMeta, sizeInBytes)
 if (lazyPruningEnabled) {
   index
 } else {
   index.filterPartitions(Nil)  // materialize all the 
partitions in memory
 }
   }
-  val partitionSchemaColumnNames = 
partitionSchema.map(_.name.toLowerCase).toSet
-  val dataSchema =
-StructType(metastoreSchema
-  .filterNot(field => 
partitionSchemaColumnNames.contains(field.name.toLowerCase)))
+  val dataSchema = relation.tableMeta.dataSchema
 
-  val relation = HadoopFsRelation(
+  val fs = HadoopFsRelation(
 location = fileIndex,
 partitionSchema = partitionSchema,
 dataSchema = dataSchema,
-bucketSpec = bucketSpec,
-fileFormat = defaultSource,
+// We don't support hive bucketed tables, only ones we write 
out.
+bucketSpec = None,
+fileFormat = fileFormatClass.newInstance(),
 options = options)(sparkSession = sparkSession)
 
-  val created = LogicalRelation(relation,
-catalogTable = Some(metastoreRelation.catalogTable))
+  val created = LogicalRelation(fs, catalogTable = 
Some(relation.tableMeta))
   tableRelationCache.put(tableIdentifier, created)
   created
 }
 
 logicalRelation
   })
 } else {
-  val rootPath = metastoreRelation.hiveQlTable.getDataLocation
+  val rootPath = tablePath
   withTableCreationLock(tableIdentifier, {
-val cached = getCached(tableIdentifier,
+val cached = getCached(
+  tableIdentifier,
   Seq(rootPath),
-  metastoreRelation,
   metastoreSchema,
   fileFormatClass,
-  bucketSpec,
   None)
 val logicalRelation = cached.getOrElse {
   val created =
 LogicalRelation(
   DataSource(
 sparkSession = sparkSession,
 paths = rootPath.toString :: Nil,
-userSpecifiedSchema = Some(metastoreRelation.schema),
-bucketSpec = bucketSpec,
+userSpecifiedSchema = Some(metastoreSchema),
+bucketSpec = None,
--- End diff --

Like what we did above, adding the same comment? 
```
// We don't support hive bucketed tables, only ones we write out.
```


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102817276
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -155,66 +153,58 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
 val cached = getCached(
   tableIdentifier,
   rootPaths,
-  metastoreRelation,
   metastoreSchema,
   fileFormatClass,
-  bucketSpec,
   Some(partitionSchema))
 
 val logicalRelation = cached.getOrElse {
-  val sizeInBytes =
-
metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
+  val sizeInBytes = 
relation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
   val fileIndex = {
-val index = new CatalogFileIndex(
-  sparkSession, metastoreRelation.catalogTable, sizeInBytes)
+val index = new CatalogFileIndex(sparkSession, 
relation.tableMeta, sizeInBytes)
 if (lazyPruningEnabled) {
   index
 } else {
   index.filterPartitions(Nil)  // materialize all the 
partitions in memory
 }
   }
-  val partitionSchemaColumnNames = 
partitionSchema.map(_.name.toLowerCase).toSet
-  val dataSchema =
-StructType(metastoreSchema
-  .filterNot(field => 
partitionSchemaColumnNames.contains(field.name.toLowerCase)))
+  val dataSchema = relation.tableMeta.dataSchema
 
-  val relation = HadoopFsRelation(
+  val fs = HadoopFsRelation(
--- End diff --

I found, normally, we use `fs` for the file system. How about changing it 
to `fsRelation`?


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102816142
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -89,7 +90,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
 val useCached =
   relation.location.rootPaths.toSet == pathsInMetastore.toSet 
&&
 logical.schema.sameType(schemaInMetastore) &&
-relation.bucketSpec == expectedBucketSpec &&
+relation.bucketSpec.isEmpty &&
--- End diff --

We might need a comment to explain it. (We don't support hive bucketed 
tables. This function `getCached ` is only used for converting Hive tables to 
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102791905
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
 ---
@@ -43,6 +43,13 @@ class SessionCatalogSuite extends PlanTest {
 
   import utils._
 
+  private def tableRelation(tableMeta: CatalogTable): CatalogRelation = {
--- End diff --

Useless?


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102611404
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -349,36 +350,44 @@ object CatalogTypes {
 
 
 /**
- * An interface that is implemented by logical plans to return the 
underlying catalog table.
- * If we can in the future consolidate SimpleCatalogRelation and 
MetastoreRelation, we should
- * probably remove this interface.
+ * A [[LogicalPlan]] that represents a table.
  */
-trait CatalogRelation {
-  def catalogTable: CatalogTable
-  def output: Seq[Attribute]
-}
+case class CatalogRelation(
+tableMeta: CatalogTable,
+dataCols: Seq[Attribute],
+partitionCols: Seq[Attribute]) extends LeafNode with 
MultiInstanceRelation {
+  assert(tableMeta.identifier.database.isDefined)
 
+  // The partition column should always appear after data columns.
+  override def output: Seq[Attribute] = dataCols ++ partitionCols
 
-/**
- * A [[LogicalPlan]] that wraps [[CatalogTable]].
- *
- * Note that in the future we should consolidate this and 
HiveCatalogRelation.
- */
-case class SimpleCatalogRelation(
-metadata: CatalogTable)
-  extends LeafNode with CatalogRelation {
-
-  override def catalogTable: CatalogTable = metadata
-
-  override lazy val resolved: Boolean = false
-
-  override val output: Seq[Attribute] = {
-val (partCols, dataCols) = metadata.schema.toAttributes
-  // Since data can be dumped in randomly with no validation, 
everything is nullable.
-  
.map(_.withNullability(true).withQualifier(Some(metadata.identifier.table)))
-  .partition { a =>
-metadata.partitionColumnNames.contains(a.name)
-  }
-dataCols ++ partCols
+  def isPartitioned: Boolean = partitionCols.nonEmpty
+
+  override def equals(relation: Any): Boolean = relation match {
+case other: CatalogRelation => tableMeta == other.tableMeta && output 
== other.output
+case _ => false
+  }
+
+  override def hashCode(): Int = {
+Objects.hashCode(tableMeta.identifier, output)
   }
+
+  /** Only compare table identifier. */
+  override def sameResult(plan: LogicalPlan): Boolean = {
+plan.canonicalized match {
+  case other: CatalogRelation => tableMeta.identifier == 
other.tableMeta.identifier
+  case _ => false
+}
+  }
--- End diff --

```Scala
override lazy val cleanArgs: Seq[Any] = Seq(tableMeta)
```

We did the same thing in the `LogicalRelation`. I think we need to do the 
same thing here. Then, we do not need to implement `sameResult` for 
`SubqueryAlias`. We always removed `SubqueryAlias` in `sameResult`


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102550705
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -594,7 +594,12 @@ class SessionCatalog(
 child = parser.parsePlan(viewText))
   SubqueryAlias(table, child, Some(name.copy(table = table, 
database = Some(db
 } else {
-  SubqueryAlias(table, SimpleCatalogRelation(metadata), None)
+  val tableRelation = CatalogRelation(
+metadata,
+// we assume the data columns are always nullable.
+metadata.dataSchema.asNullable.toAttributes,
+metadata.partitionSchema.toAttributes)
--- End diff --

Do we need to call `asNullable` for partitionSchema? In the original 
`SimpleCatalogRelation`, we did it for `output`


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102545703
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -349,36 +350,44 @@ object CatalogTypes {
 
 
 /**
- * An interface that is implemented by logical plans to return the 
underlying catalog table.
- * If we can in the future consolidate SimpleCatalogRelation and 
MetastoreRelation, we should
- * probably remove this interface.
+ * A [[LogicalPlan]] that represents a table.
  */
-trait CatalogRelation {
-  def catalogTable: CatalogTable
-  def output: Seq[Attribute]
-}
+case class CatalogRelation(
+tableMeta: CatalogTable,
+dataCols: Seq[Attribute],
+partitionCols: Seq[Attribute]) extends LeafNode with 
MultiInstanceRelation {
+  assert(tableMeta.identifier.database.isDefined)
--- End diff --

Add two more asserts?
```Scala
  assert(tableMeta.partitionSchema == 
StructType.fromAttributes(partitionCols))
  assert(tableMeta.dataSchema == StructType.fromAttributes(dataCols))
```


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

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

https://github.com/apache/spark/pull/17015#discussion_r102360866
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -349,36 +350,42 @@ object CatalogTypes {
 
 
 /**
- * An interface that is implemented by logical plans to return the 
underlying catalog table.
- * If we can in the future consolidate SimpleCatalogRelation and 
MetastoreRelation, we should
- * probably remove this interface.
+ * A [[LogicalPlan]] that represents a table.
  */
-trait CatalogRelation {
-  def catalogTable: CatalogTable
-  def output: Seq[Attribute]
-}
+case class CatalogRelation(
+tableMeta: CatalogTable,
+dataCols: Seq[Attribute],
+partitionCols: Seq[Attribute]) extends LeafNode with 
MultiInstanceRelation {
--- End diff --

yes, and at least we need a `Seq[Attribute]` as output.


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-21 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102338672
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -349,36 +350,42 @@ object CatalogTypes {
 
 
 /**
- * An interface that is implemented by logical plans to return the 
underlying catalog table.
- * If we can in the future consolidate SimpleCatalogRelation and 
MetastoreRelation, we should
- * probably remove this interface.
+ * A [[LogicalPlan]] that represents a table.
  */
-trait CatalogRelation {
-  def catalogTable: CatalogTable
-  def output: Seq[Attribute]
-}
+case class CatalogRelation(
+tableMeta: CatalogTable,
+dataCols: Seq[Attribute],
+partitionCols: Seq[Attribute]) extends LeafNode with 
MultiInstanceRelation {
--- End diff --

`dataCols` and `partitionCols` are needed only because `CatalogRelation` 
extends `MultiInstanceRelation`? 


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-21 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/17015#discussion_r102303660
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -349,36 +350,42 @@ object CatalogTypes {
 
 
 /**
- * An interface that is implemented by logical plans to return the 
underlying catalog table.
- * If we can in the future consolidate SimpleCatalogRelation and 
MetastoreRelation, we should
- * probably remove this interface.
+ * A [[LogicalPlan]] that represents a table relation.
  */
-trait CatalogRelation {
-  def catalogTable: CatalogTable
-  def output: Seq[Attribute]
-}
+case class CatalogTableRelation(
--- End diff --

Conceptually, table and relation are the same. How about keeping the 
original name `CatalogRelation`?


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

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

https://github.com/apache/spark/pull/17015#discussion_r102302366
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala
 ---
@@ -138,27 +153,35 @@ case class HiveTableScanExec(
 }
   }
 
+  // exposed for tests
+  @transient lazy val rawPartitions = {
+val prunedPartitions = if 
(sparkSession.sessionState.conf.metastorePartitionPruning) {
--- End diff --

do we still need this? data source tables always do metastore partition 
pruning.


---
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 #17015: [SPARK-19678][SQL] remove MetastoreRelation

2017-02-21 Thread cloud-fan
GitHub user cloud-fan opened a pull request:

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

[SPARK-19678][SQL] remove MetastoreRelation

## What changes were proposed in this pull request?

`MetastoreRelation` is used to represent table relation for hive tables, 
and provides some hive related information. We will resolve 
`SimpleCatalogRelation` to `MetastoreRelation` for hive tables, which is 
unnecessary as these 2 are the same essentially. This PR merges 
`SimpleCatalogRelation` and `MetastoreRelation` 

## 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 table-relation

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

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


commit 483fceeefda3e24adc5691827f6c0994791f1e27
Author: Wenchen Fan 
Date:   2017-02-21T10:24:37Z

remove MetastoreRelation




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