Skip to content

Commit 772ddbe

Browse files
Michael Allmancloud-fan
authored andcommitted
[SPARK-18572][SQL] Add a method listPartitionNames to ExternalCatalog
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-18572) ## What changes were proposed in this pull request? Currently Spark answers the `SHOW PARTITIONS` command by fetching all of the table's partition metadata from the external catalog and constructing partition names therefrom. The Hive client has a `getPartitionNames` method which is many times faster for this purpose, with the performance improvement scaling with the number of partitions in a table. To test the performance impact of this PR, I ran the `SHOW PARTITIONS` command on two Hive tables with large numbers of partitions. One table has ~17,800 partitions, and the other has ~95,000 partitions. For the purposes of this PR, I'll call the former table `table1` and the latter table `table2`. I ran 5 trials for each table with before-and-after versions of this PR. The results are as follows: Spark at bdc8153, `SHOW PARTITIONS table1`, times in seconds: 7.901 3.983 4.018 4.331 4.261 Spark at bdc8153, `SHOW PARTITIONS table2` (Timed out after 10 minutes with a `SocketTimeoutException`.) Spark at this PR, `SHOW PARTITIONS table1`, times in seconds: 3.801 0.449 0.395 0.348 0.336 Spark at this PR, `SHOW PARTITIONS table2`, times in seconds: 5.184 1.63 1.474 1.519 1.41 Taking the best times from each trial, we get a 12x performance improvement for a table with ~17,800 partitions and at least a 426x improvement for a table with ~95,000 partitions. More significantly, the latter command doesn't even complete with the current code in master. This is actually a patch we've been using in-house at VideoAmp since Spark 1.1. It's made all the difference in the practical usability of our largest tables. Even with tables with about 1,000 partitions there's a performance improvement of about 2-3x. ## How was this patch tested? I added a unit test to `VersionsSuite` which tests that the Hive client's `getPartitionNames` method returns the correct number of partitions. Author: Michael Allman <[email protected]> Closes #15998 from mallman/spark-18572-list_partition_names.
1 parent 4af142f commit 772ddbe

File tree

12 files changed

+221
-30
lines changed

12 files changed

+221
-30
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala

Lines changed: 24 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -189,15 +189,37 @@ abstract class ExternalCatalog {
189189
table: String,
190190
spec: TablePartitionSpec): Option[CatalogTablePartition]
191191

192+
/**
193+
* List the names of all partitions that belong to the specified table, assuming it exists.
194+
*
195+
* For a table with partition columns p1, p2, p3, each partition name is formatted as
196+
* `p1=v1/p2=v2/p3=v3`. Each partition column name and value is an escaped path name, and can be
197+
* decoded with the `ExternalCatalogUtils.unescapePathName` method.
198+
*
199+
* The returned sequence is sorted as strings.
200+
*
201+
* A partial partition spec may optionally be provided to filter the partitions returned, as
202+
* described in the `listPartitions` method.
203+
*
204+
* @param db database name
205+
* @param table table name
206+
* @param partialSpec partition spec
207+
*/
208+
def listPartitionNames(
209+
db: String,
210+
table: String,
211+
partialSpec: Option[TablePartitionSpec] = None): Seq[String]
212+
192213
/**
193214
* List the metadata of all partitions that belong to the specified table, assuming it exists.
194215
*
195216
* A partial partition spec may optionally be provided to filter the partitions returned.
196217
* For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'),
197218
* then a partial spec of (a='1') will return the first two only.
219+
*
198220
* @param db database name
199221
* @param table table name
200-
* @param partialSpec partition spec
222+
* @param partialSpec partition spec
201223
*/
202224
def listPartitions(
203225
db: String,
@@ -210,7 +232,7 @@ abstract class ExternalCatalog {
210232
*
211233
* @param db database name
212234
* @param table table name
213-
* @param predicates partition-pruning predicates
235+
* @param predicates partition-pruning predicates
214236
*/
215237
def listPartitionsByFilter(
216238
db: String,

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ import org.apache.spark.{SparkConf, SparkException}
2828
import org.apache.spark.sql.AnalysisException
2929
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
3030
import org.apache.spark.sql.catalyst.analysis._
31+
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName
3132
import org.apache.spark.sql.catalyst.expressions.Expression
3233
import org.apache.spark.sql.catalyst.util.StringUtils
3334

@@ -488,6 +489,19 @@ class InMemoryCatalog(
488489
}
489490
}
490491

492+
override def listPartitionNames(
493+
db: String,
494+
table: String,
495+
partialSpec: Option[TablePartitionSpec] = None): Seq[String] = synchronized {
496+
val partitionColumnNames = getTable(db, table).partitionColumnNames
497+
498+
listPartitions(db, table, partialSpec).map { partition =>
499+
partitionColumnNames.map { name =>
500+
escapePathName(name) + "=" + escapePathName(partition.spec(name))
501+
}.mkString("/")
502+
}.sorted
503+
}
504+
491505
override def listPartitions(
492506
db: String,
493507
table: String,

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -748,6 +748,26 @@ class SessionCatalog(
748748
externalCatalog.getPartition(db, table, spec)
749749
}
750750

751+
/**
752+
* List the names of all partitions that belong to the specified table, assuming it exists.
753+
*
754+
* A partial partition spec may optionally be provided to filter the partitions returned.
755+
* For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'),
756+
* then a partial spec of (a='1') will return the first two only.
757+
*/
758+
def listPartitionNames(
759+
tableName: TableIdentifier,
760+
partialSpec: Option[TablePartitionSpec] = None): Seq[String] = {
761+
val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase))
762+
val table = formatTableName(tableName.table)
763+
requireDbExists(db)
764+
requireTableExists(TableIdentifier(table, Option(db)))
765+
partialSpec.foreach { spec =>
766+
requirePartialMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName))
767+
}
768+
externalCatalog.listPartitionNames(db, table, partialSpec)
769+
}
770+
751771
/**
752772
* List the metadata of all partitions that belong to the specified table, assuming it exists.
753773
*
@@ -762,6 +782,9 @@ class SessionCatalog(
762782
val table = formatTableName(tableName.table)
763783
requireDbExists(db)
764784
requireTableExists(TableIdentifier(table, Option(db)))
785+
partialSpec.foreach { spec =>
786+
requirePartialMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName))
787+
}
765788
externalCatalog.listPartitions(db, table, partialSpec)
766789
}
767790

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -346,6 +346,31 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac
346346
assert(new Path(partitionLocation) == defaultPartitionLocation)
347347
}
348348

349+
test("list partition names") {
350+
val catalog = newBasicCatalog()
351+
val newPart = CatalogTablePartition(Map("a" -> "1", "b" -> "%="), storageFormat)
352+
catalog.createPartitions("db2", "tbl2", Seq(newPart), ignoreIfExists = false)
353+
354+
val partitionNames = catalog.listPartitionNames("db2", "tbl2")
355+
assert(partitionNames == Seq("a=1/b=%25%3D", "a=1/b=2", "a=3/b=4"))
356+
}
357+
358+
test("list partition names with partial partition spec") {
359+
val catalog = newBasicCatalog()
360+
val newPart = CatalogTablePartition(Map("a" -> "1", "b" -> "%="), storageFormat)
361+
catalog.createPartitions("db2", "tbl2", Seq(newPart), ignoreIfExists = false)
362+
363+
val partitionNames1 = catalog.listPartitionNames("db2", "tbl2", Some(Map("a" -> "1")))
364+
assert(partitionNames1 == Seq("a=1/b=%25%3D", "a=1/b=2"))
365+
366+
// Partial partition specs including "weird" partition values should use the unescaped values
367+
val partitionNames2 = catalog.listPartitionNames("db2", "tbl2", Some(Map("b" -> "%=")))
368+
assert(partitionNames2 == Seq("a=1/b=%25%3D"))
369+
370+
val partitionNames3 = catalog.listPartitionNames("db2", "tbl2", Some(Map("b" -> "%25%3D")))
371+
assert(partitionNames3.isEmpty)
372+
}
373+
349374
test("list partitions with partial partition spec") {
350375
val catalog = newBasicCatalog()
351376
val parts = catalog.listPartitions("db2", "tbl2", Some(Map("a" -> "1")))

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala

Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -878,6 +878,31 @@ class SessionCatalogSuite extends SparkFunSuite {
878878
"the partition spec (a, b) defined in table '`db2`.`tbl1`'"))
879879
}
880880

881+
test("list partition names") {
882+
val catalog = new SessionCatalog(newBasicCatalog())
883+
val expectedPartitionNames = Seq("a=1/b=2", "a=3/b=4")
884+
assert(catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2"))) ==
885+
expectedPartitionNames)
886+
// List partition names without explicitly specifying database
887+
catalog.setCurrentDatabase("db2")
888+
assert(catalog.listPartitionNames(TableIdentifier("tbl2")) == expectedPartitionNames)
889+
}
890+
891+
test("list partition names with partial partition spec") {
892+
val catalog = new SessionCatalog(newBasicCatalog())
893+
assert(
894+
catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")), Some(Map("a" -> "1"))) ==
895+
Seq("a=1/b=2"))
896+
}
897+
898+
test("list partition names with invalid partial partition spec") {
899+
val catalog = new SessionCatalog(newBasicCatalog())
900+
intercept[AnalysisException] {
901+
catalog.listPartitionNames(TableIdentifier("tbl2", Some("db2")),
902+
Some(Map("unknown" -> "unknown")))
903+
}
904+
}
905+
881906
test("list partitions") {
882907
val catalog = new SessionCatalog(newBasicCatalog())
883908
assert(catalogPartitionsEqual(
@@ -887,6 +912,20 @@ class SessionCatalogSuite extends SparkFunSuite {
887912
assert(catalogPartitionsEqual(catalog.listPartitions(TableIdentifier("tbl2")), part1, part2))
888913
}
889914

915+
test("list partitions with partial partition spec") {
916+
val catalog = new SessionCatalog(newBasicCatalog())
917+
assert(catalogPartitionsEqual(
918+
catalog.listPartitions(TableIdentifier("tbl2", Some("db2")), Some(Map("a" -> "1"))), part1))
919+
}
920+
921+
test("list partitions with invalid partial partition spec") {
922+
val catalog = new SessionCatalog(newBasicCatalog())
923+
intercept[AnalysisException] {
924+
catalog.listPartitions(
925+
TableIdentifier("tbl2", Some("db2")), Some(Map("unknown" -> "unknown")))
926+
}
927+
}
928+
890929
test("list partitions when database/table does not exist") {
891930
val catalog = new SessionCatalog(newBasicCatalog())
892931
intercept[NoSuchDatabaseException] {

sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala

Lines changed: 1 addition & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -729,13 +729,6 @@ case class ShowPartitionsCommand(
729729
AttributeReference("partition", StringType, nullable = false)() :: Nil
730730
}
731731

732-
private def getPartName(spec: TablePartitionSpec, partColNames: Seq[String]): String = {
733-
partColNames.map { name =>
734-
ExternalCatalogUtils.escapePathName(name) + "=" +
735-
ExternalCatalogUtils.escapePathName(spec(name))
736-
}.mkString(File.separator)
737-
}
738-
739732
override def run(sparkSession: SparkSession): Seq[Row] = {
740733
val catalog = sparkSession.sessionState.catalog
741734
val table = catalog.getTableMetadata(tableName)
@@ -772,10 +765,7 @@ case class ShowPartitionsCommand(
772765
}
773766
}
774767

775-
val partNames = catalog.listPartitions(tableName, spec).map { p =>
776-
getPartName(p.spec, table.partitionColumnNames)
777-
}
778-
768+
val partNames = catalog.listPartitionNames(tableName, spec)
779769
partNames.map(Row(_))
780770
}
781771
}

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -161,8 +161,8 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] {
161161
insert.copy(partition = parts.map(p => (p._1, None)), child = Project(projectList, query))
162162

163163

164-
case i @ logical.InsertIntoTable(
165-
l @ LogicalRelation(t: HadoopFsRelation, _, table), part, query, overwrite, false)
164+
case logical.InsertIntoTable(
165+
l @ LogicalRelation(t: HadoopFsRelation, _, table), _, query, overwrite, false)
166166
if query.resolved && t.schema.sameType(query.schema) =>
167167

168168
// Sanity checks
@@ -192,11 +192,19 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] {
192192
var initialMatchingPartitions: Seq[TablePartitionSpec] = Nil
193193
var customPartitionLocations: Map[TablePartitionSpec, String] = Map.empty
194194

195+
val staticPartitionKeys: TablePartitionSpec = if (overwrite.enabled) {
196+
overwrite.staticPartitionKeys.map { case (k, v) =>
197+
(partitionSchema.map(_.name).find(_.equalsIgnoreCase(k)).get, v)
198+
}
199+
} else {
200+
Map.empty
201+
}
202+
195203
// When partitions are tracked by the catalog, compute all custom partition locations that
196204
// may be relevant to the insertion job.
197205
if (partitionsTrackedByCatalog) {
198206
val matchingPartitions = t.sparkSession.sessionState.catalog.listPartitions(
199-
l.catalogTable.get.identifier, Some(overwrite.staticPartitionKeys))
207+
l.catalogTable.get.identifier, Some(staticPartitionKeys))
200208
initialMatchingPartitions = matchingPartitions.map(_.spec)
201209
customPartitionLocations = getCustomPartitionLocations(
202210
t.sparkSession, l.catalogTable.get, outputPath, matchingPartitions)
@@ -225,14 +233,6 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] {
225233
t.location.refresh()
226234
}
227235

228-
val staticPartitionKeys: TablePartitionSpec = if (overwrite.enabled) {
229-
overwrite.staticPartitionKeys.map { case (k, v) =>
230-
(partitionSchema.map(_.name).find(_.equalsIgnoreCase(k)).get, v)
231-
}
232-
} else {
233-
Map.empty
234-
}
235-
236236
val insertCmd = InsertIntoHadoopFsRelationCommand(
237237
outputPath,
238238
staticPartitionKeys,

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -244,13 +244,22 @@ object PartitioningUtils {
244244

245245
/**
246246
* Given a partition path fragment, e.g. `fieldOne=1/fieldTwo=2`, returns a parsed spec
247-
* for that fragment, e.g. `Map(("fieldOne", "1"), ("fieldTwo", "2"))`.
247+
* for that fragment as a `TablePartitionSpec`, e.g. `Map(("fieldOne", "1"), ("fieldTwo", "2"))`.
248248
*/
249249
def parsePathFragment(pathFragment: String): TablePartitionSpec = {
250+
parsePathFragmentAsSeq(pathFragment).toMap
251+
}
252+
253+
/**
254+
* Given a partition path fragment, e.g. `fieldOne=1/fieldTwo=2`, returns a parsed spec
255+
* for that fragment as a `Seq[(String, String)]`, e.g.
256+
* `Seq(("fieldOne", "1"), ("fieldTwo", "2"))`.
257+
*/
258+
def parsePathFragmentAsSeq(pathFragment: String): Seq[(String, String)] = {
250259
pathFragment.split("/").map { kv =>
251260
val pair = kv.split("=", 2)
252261
(unescapePathName(pair(0)), unescapePathName(pair(1)))
253-
}.toMap
262+
}
254263
}
255264

256265
/**

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala

Lines changed: 38 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -35,10 +35,12 @@ import org.apache.spark.sql.AnalysisException
3535
import org.apache.spark.sql.catalyst.TableIdentifier
3636
import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
3737
import org.apache.spark.sql.catalyst.catalog._
38+
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName
3839
import org.apache.spark.sql.catalyst.expressions._
3940
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Statistics}
4041
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
4142
import org.apache.spark.sql.execution.command.DDLUtils
43+
import org.apache.spark.sql.execution.datasources.PartitioningUtils
4244
import org.apache.spark.sql.hive.client.HiveClient
4345
import org.apache.spark.sql.internal.HiveSerDe
4446
import org.apache.spark.sql.internal.StaticSQLConf._
@@ -812,9 +814,21 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
812814
spec.map { case (k, v) => k.toLowerCase -> v }
813815
}
814816

817+
// Build a map from lower-cased partition column names to exact column names for a given table
818+
private def buildLowerCasePartColNameMap(table: CatalogTable): Map[String, String] = {
819+
val actualPartColNames = table.partitionColumnNames
820+
actualPartColNames.map(colName => (colName.toLowerCase, colName)).toMap
821+
}
822+
815823
// Hive metastore is not case preserving and the column names of the partition specification we
816824
// get from the metastore are always lower cased. We should restore them w.r.t. the actual table
817825
// partition columns.
826+
private def restorePartitionSpec(
827+
spec: TablePartitionSpec,
828+
partColMap: Map[String, String]): TablePartitionSpec = {
829+
spec.map { case (k, v) => partColMap(k.toLowerCase) -> v }
830+
}
831+
818832
private def restorePartitionSpec(
819833
spec: TablePartitionSpec,
820834
partCols: Seq[String]): TablePartitionSpec = {
@@ -927,13 +941,32 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
927941
/**
928942
* Returns the partition names from hive metastore for a given table in a database.
929943
*/
944+
override def listPartitionNames(
945+
db: String,
946+
table: String,
947+
partialSpec: Option[TablePartitionSpec] = None): Seq[String] = withClient {
948+
val catalogTable = getTable(db, table)
949+
val partColNameMap = buildLowerCasePartColNameMap(catalogTable).mapValues(escapePathName)
950+
val clientPartitionNames =
951+
client.getPartitionNames(catalogTable, partialSpec.map(lowerCasePartitionSpec))
952+
clientPartitionNames.map { partName =>
953+
val partSpec = PartitioningUtils.parsePathFragmentAsSeq(partName)
954+
partSpec.map { case (partName, partValue) =>
955+
partColNameMap(partName.toLowerCase) + "=" + escapePathName(partValue)
956+
}.mkString("/")
957+
}
958+
}
959+
960+
/**
961+
* Returns the partitions from hive metastore for a given table in a database.
962+
*/
930963
override def listPartitions(
931964
db: String,
932965
table: String,
933966
partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = withClient {
934-
val actualPartColNames = getTable(db, table).partitionColumnNames
967+
val partColNameMap = buildLowerCasePartColNameMap(getTable(db, table))
935968
client.getPartitions(db, table, partialSpec.map(lowerCasePartitionSpec)).map { part =>
936-
part.copy(spec = restorePartitionSpec(part.spec, actualPartColNames))
969+
part.copy(spec = restorePartitionSpec(part.spec, partColNameMap))
937970
}
938971
}
939972

@@ -954,10 +987,11 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
954987
}
955988

956989
val partitionSchema = catalogTable.partitionSchema
990+
val partColNameMap = buildLowerCasePartColNameMap(getTable(db, table))
957991

958992
if (predicates.nonEmpty) {
959993
val clientPrunedPartitions = client.getPartitionsByFilter(rawTable, predicates).map { part =>
960-
part.copy(spec = restorePartitionSpec(part.spec, catalogTable.partitionColumnNames))
994+
part.copy(spec = restorePartitionSpec(part.spec, partColNameMap))
961995
}
962996
val boundPredicate =
963997
InterpretedPredicate.create(predicates.reduce(And).transform {
@@ -968,7 +1002,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
9681002
clientPrunedPartitions.filter { p => boundPredicate(p.toRow(partitionSchema)) }
9691003
} else {
9701004
client.getPartitions(catalogTable).map { part =>
971-
part.copy(spec = restorePartitionSpec(part.spec, catalogTable.partitionColumnNames))
1005+
part.copy(spec = restorePartitionSpec(part.spec, partColNameMap))
9721006
}
9731007
}
9741008
}

0 commit comments

Comments
 (0)