Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[spark] show table extended #4603

Merged
merged 3 commits into from
Dec 3, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 11 additions & 0 deletions docs/content/spark/auxiliary.md
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,17 @@ SHOW PARTITIONS my_table;
SHOW PARTITIONS my_table PARTITION (dt=20230817);
```

## Show Table Extended
The SHOW TABLE EXTENDED statement is used to list table or partition information.

```sql
-- Lists tables that satisfy regular expressions
SHOW TABLE EXTENDED IN db_name LIKE 'test*';

-- Lists the specified partition information for the table
SHOW TABLE EXTENDED IN db_name LIKE 'table_name' PARTITION(pt = '2024');
```

## Analyze table

The ANALYZE TABLE statement collects statistics about the table, that are to be used by the query optimizer to find a better query execution plan.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement {
}

override def loadPartitionMetadata(ident: InternalRow): JMap[String, String] = {
throw new UnsupportedOperationException("Load partition is not supported")
Map.empty[String, String].asJava
}

override def listPartitionIdentifiers(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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.paimon.spark.catalyst.analysis

import org.apache.spark.sql.PaimonUtils.{normalizePartitionSpec, requireExactMatchedPartitionSpec}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{PartitionSpec, ResolvedPartitionSpec, UnresolvedPartitionSpec}
import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec.conf
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog}
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._
import org.apache.spark.sql.types.{StringType, StructField, StructType}

object PaimonResolvePartitionSpec {

def resolve(
catalog: TableCatalog,
tableIndent: Identifier,
partitionSpec: PartitionSpec): ResolvedPartitionSpec = {
val table = catalog.loadTable(tableIndent).asPartitionable
partitionSpec match {
case u: UnresolvedPartitionSpec =>
val partitionSchema = table.partitionSchema()
resolvePartitionSpec(table.name(), u, partitionSchema, allowPartitionSpec = false)
case o => o.asInstanceOf[ResolvedPartitionSpec]
}
}

private def resolvePartitionSpec(
tableName: String,
partSpec: UnresolvedPartitionSpec,
partSchema: StructType,
allowPartitionSpec: Boolean): ResolvedPartitionSpec = {
val normalizedSpec = normalizePartitionSpec(partSpec.spec, partSchema, tableName, conf.resolver)
if (!allowPartitionSpec) {
requireExactMatchedPartitionSpec(tableName, normalizedSpec, partSchema.fieldNames)
}
val partitionNames = normalizedSpec.keySet
val requestedFields = partSchema.filter(field => partitionNames.contains(field.name))
ResolvedPartitionSpec(
requestedFields.map(_.name),
convertToPartIdent(normalizedSpec, requestedFields),
partSpec.location)
}

def convertToPartIdent(
partitionSpec: TablePartitionSpec,
schema: Seq[StructField]): InternalRow = {
val partValues = schema.map {
part =>
val raw = partitionSpec.get(part.name).orNull
val dt = CharVarcharUtils.replaceCharVarcharWithString(part.dataType)
Cast(Literal.create(raw, StringType), dt, Some(conf.sessionLocalTimeZone)).eval()
}
InternalRow.fromSeq(partValues)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@ class PaimonSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
extensions.injectResolutionRule(spark => new PaimonAnalysis(spark))
extensions.injectResolutionRule(spark => PaimonProcedureResolver(spark))
extensions.injectResolutionRule(spark => PaimonViewResolver(spark))
extensions.injectResolutionRule(
spark => SparkShimLoader.getSparkShim.createCustomResolution(spark))
extensions.injectResolutionRule(spark => PaimonIncompatibleResolutionRules(spark))

extensions.injectPostHocResolutionRule(spark => PaimonPostHocResolutionRules(spark))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,15 @@ package org.apache.spark.sql

import org.apache.spark.executor.OutputMetrics
import org.apache.spark.rdd.InputFileBlockHolder
import org.apache.spark.sql.catalyst.analysis.Resolver
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.connector.expressions.{FieldReference, NamedReference}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.PartitioningUtils
import org.apache.spark.util.{Utils => SparkUtils}

/**
Expand Down Expand Up @@ -87,4 +91,19 @@ object PaimonUtils {
outputMetrics.setBytesWritten(bytesWritten)
outputMetrics.setRecordsWritten(recordsWritten)
}

def normalizePartitionSpec[T](
partitionSpec: Map[String, T],
partCols: StructType,
tblName: String,
resolver: Resolver): Map[String, T] = {
PartitioningUtils.normalizePartitionSpec(partitionSpec, partCols, tblName, resolver)
}

def requireExactMatchedPartitionSpec(
tableName: String,
spec: TablePartitionSpec,
partitionColumnNames: Seq[String]): Unit = {
PartitioningUtils.requireExactMatchedPartitionSpec(tableName, spec, partitionColumnNames)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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.connector.catalog

object PaimonCatalogImplicits {

import CatalogV2Implicits._

implicit class PaimonCatalogHelper(plugin: CatalogPlugin) extends CatalogHelper(plugin)

implicit class PaimonNamespaceHelper(namespace: Array[String]) extends NamespaceHelper(namespace)

// implicit class PaimonTableHelper(table: Table) extends TableHelper(table)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This line can be deleted.

}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.spark.SparkConf
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.catalog.ExternalCatalog
import org.apache.spark.sql.connector.catalog.CatalogV2Util
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
import org.apache.spark.sql.paimon.ReflectUtils

Expand All @@ -40,4 +41,6 @@ object PaimonCatalogUtils {
hadoopConf)
}

val TABLE_RESERVED_PROPERTIES: Seq[String] = CatalogV2Util.TABLE_RESERVED_PROPERTIES

}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@ import org.apache.paimon.types.{DataType, RowType}
import org.apache.spark.sql.{Column, SparkSession}
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog}
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.types.StructType
Expand All @@ -39,6 +41,8 @@ trait SparkShim {

def createSparkParser(delegate: ParserInterface): ParserInterface

def createCustomResolution(spark: SparkSession): Rule[LogicalPlan]

def createSparkInternalRow(rowType: RowType): SparkInternalRow

def createSparkArrayData(elementType: DataType): SparkArrayData
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,76 @@ import java.util.Objects

class DescribeTableTest extends PaimonSparkTestBase {

test("Paimon show: show table extended") {
val testDB = "test_show"
withDatabase(testDB) {
spark.sql("CREATE TABLE s1 (id INT)")

spark.sql(s"CREATE DATABASE $testDB")
spark.sql(s"USE $testDB")
spark.sql("CREATE TABLE s2 (id INT, pt STRING) PARTITIONED BY (pt)")
spark.sql("CREATE TABLE s3 (id INT, pt1 STRING, pt2 STRING) PARTITIONED BY (pt1, pt2)")

spark.sql("INSERT INTO s2 VALUES (1, '2024'), (2, '2024'), (3, '2025'), (4, '2026')")
spark.sql("""
|INSERT INTO s3
|VALUES
|(1, '2024', '11'), (2, '2024', '12'), (3, '2025', '11'), (4, '2025', '12')
|""".stripMargin)

// SHOW TABL EXTENDED will give four columns: namespace, tableName, isTemporary, information.
checkAnswer(
sql(s"SHOW TABLE EXTENDED IN $dbName0 LIKE '*'")
.select("namespace", "tableName", "isTemporary"),
Row("test", "s1", false))
checkAnswer(
sql(s"SHOW TABLE EXTENDED IN $testDB LIKE '*'")
.select("namespace", "tableName", "isTemporary"),
Row(testDB, "s2", false) :: Row(testDB, "s3", false) :: Nil
)

// check table s1
val res1 = spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's2'").select("information")
Assertions.assertEquals(1, res1.count())
val information1 = res1
.collect()
.head
.getString(0)
.split("\n")
.map {
line =>
val kv = line.split(": ", 2)
kv(0) -> kv(1)
}
.toMap
Assertions.assertEquals(information1("Catalog"), "paimon")
Assertions.assertEquals(information1("Namespace"), testDB)
Assertions.assertEquals(information1("Table"), "s2")
Assertions.assertEquals(information1("Provider"), "paimon")
Assertions.assertEquals(information1("Location"), loadTable(testDB, "s2").location().toString)

// check table s2 partition info
val error1 = intercept[Exception] {
spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's2' PARTITION(pt='2022')")
}.getMessage
assert(error1.contains("PARTITIONS_NOT_FOUND"))

val error2 = intercept[Exception] {
spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's3' PARTITION(pt1='2024')")
}.getMessage
assert(error2.contains("Partition spec is invalid"))

val res2 =
spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's3' PARTITION(pt1 = '2024', pt2 = 11)")
checkAnswer(
res2.select("namespace", "tableName", "isTemporary"),
Row(testDB, "s3", false)
)
Assertions.assertTrue(
res2.select("information").collect().head.getString(0).contains("Partition Values"))
}
}

test(s"Paimon describe: describe table comment") {
var comment = "test comment"
spark.sql(s"""
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.paimon.spark.catalyst.analysis

import org.apache.paimon.spark.commands.{PaimonShowTablePartitionCommand, PaimonShowTablesExtendedCommand}

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.analysis.{PartitionSpec, ResolvedNamespace, UnresolvedPartitionSpec}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ShowTableExtended}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.catalog.Identifier

case class Spark3ResolutionRules(session: SparkSession)
extends Rule[LogicalPlan]
with SQLConfHelper {

import org.apache.spark.sql.connector.catalog.PaimonCatalogImplicits._

override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown {
case ShowTableExtended(
ResolvedNamespace(catalog, ns),
pattern,
partitionSpec @ (None | Some(UnresolvedPartitionSpec(_, _))),
output) =>
partitionSpec
.map {
spec: PartitionSpec =>
val table = Identifier.of(ns.toArray, pattern)
val resolvedSpec =
PaimonResolvePartitionSpec.resolve(catalog.asTableCatalog, table, spec)
PaimonShowTablePartitionCommand(output, catalog.asTableCatalog, table, resolvedSpec)
}
.getOrElse {
PaimonShowTablesExtendedCommand(catalog.asTableCatalog, ns, pattern, output)
}

}

}
Loading
Loading