Skip to content

Commit

Permalink
[SPARK-17369][SQL] MetastoreRelation toJSON throws AssertException du…
Browse files Browse the repository at this point in the history
…e to missing otherCopyArgs

## What changes were proposed in this pull request?

`TreeNode.toJSON` requires a subclass to explicitly override otherCopyArgs to include currying construction arguments, otherwise it reports AssertException telling that the construction argument values' count doesn't match the construction argument names' count.

For class `MetastoreRelation`, it has a currying construction parameter `client: HiveClient`, but Spark forgets to add it to the list of otherCopyArgs.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <[email protected]>

Closes #14928 from clockfly/metastore_relation_toJSON.
  • Loading branch information
clockfly authored and cloud-fan committed Sep 6, 2016
1 parent 8d08f43 commit afb3d5d
Show file tree
Hide file tree
Showing 2 changed files with 40 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ private[hive] case class MetastoreRelation(
Objects.hashCode(databaseName, tableName, output)
}

override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: sparkSession :: Nil
override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: client :: sparkSession :: Nil

private def toHiveColumn(c: StructField): FieldSchema = {
new FieldSchema(c.name, c.dataType.catalogString, c.getComment.orNull)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.hive

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType}
import org.apache.spark.sql.types.{IntegerType, StructField, StructType}

class MetastoreRelationSuite extends SparkFunSuite {
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, null)

// No exception should be thrown
relation.makeCopy(Array("db", "test"))
// No exception should be thrown
relation.toJSON
}
}

0 comments on commit afb3d5d

Please sign in to comment.