Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 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
Original file line number Diff line number Diff line change
Expand Up @@ -624,7 +624,7 @@ case class AlterTableRecoverPartitionsCommand(
try {
implicit val ec = ExecutionContext.fromExecutor(evalPool)
scanPartitions(spark, fs, pathFilter, root, Map(), table.partitionColumnNames, threshold,
spark.sessionState.conf.resolver)
spark.sessionState.conf.resolver, listFilesInParallel = true)
} finally {
evalPool.shutdown()
}
Expand Down Expand Up @@ -656,7 +656,10 @@ case class AlterTableRecoverPartitionsCommand(
spec: TablePartitionSpec,
partitionNames: Seq[String],
threshold: Int,
resolver: Resolver)(implicit ec: ExecutionContext): Seq[(TablePartitionSpec, Path)] = {
resolver: Resolver,
listFilesInParallel: Boolean)(implicit ec: ExecutionContext)
: Seq[(TablePartitionSpec, Path)] = {

if (partitionNames.isEmpty) {
return Seq(spec -> path)
}
Expand All @@ -671,7 +674,7 @@ case class AlterTableRecoverPartitionsCommand(
val value = ExternalCatalogUtils.unescapePathName(ps(1))
if (resolver(columnName, partitionNames.head)) {
scanPartitions(spark, fs, filter, st.getPath, spec ++ Map(partitionNames.head -> value),
partitionNames.drop(1), threshold, resolver)
partitionNames.drop(1), threshold, resolver, listFilesInParallel = false)
Copy link
Member

Choose a reason for hiding this comment

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

This change might introduce performance regression. Do you know why it works when using .par previously?

Copy link
Member Author

Choose a reason for hiding this comment

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

This change might introduce performance regression.

Right, if there is significant disbalance of sub-folders, scanning will be slower probably.

Do you know why it works when using .par previously?

Scala parallel collections can cope with nested calls. See this from slide 12: https://www.slideshare.net/AleksandarProkopec/scala-parallel-collections

@gatorsmile I can revert Scala parallel collections here since we use them on the driver, and parmap is not not necessary here.

Copy link
Member

Choose a reason for hiding this comment

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

@MaxGekk Do you have a stack trace on each thread when a dead lock occurs?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes, I do jstack.txt

Copy link
Member

Choose a reason for hiding this comment

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

Thank you attaching the stack trace. I have just looked at it. It looks strange to me. Every thread is waiting for. No blocker is there, only one locked exists.
In typical case, a deadlock occurs due to existence of blocker as attached stack trace in #22221

I will investigate it furthermore tomorrow if we need to use this implementation instead of reverting it to the original implementation to use Scala parallel collection.

...
        - parking to wait for  <0x0000000793c0d610> (a scala.concurrent.impl.Promise$CompletionLatch)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
        at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:206)
        at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:222)
        at scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:227)
        at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:220)
        at org.apache.spark.util.ThreadUtils$.parmap(ThreadUtils.scala:317)
        at org.apache.spark.sql.execution.command.AlterTableRecoverPartitionsCommand.scanPartitions(ddl.scala:690)
        at org.apache.spark.sql.execution.command.AlterTableRecoverPartitionsCommand.run(ddl.scala:626)
        at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
        - locked <0x0000000793b04e88> (a org.apache.spark.sql.execution.command.ExecutedCommandExec)
        at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
        at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
...

Copy link
Member

Choose a reason for hiding this comment

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

Does it mean there is no avaiable thread in a given thread pool when a problem try to execute a new Future?

Copy link
Member Author

Choose a reason for hiding this comment

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

@kiszk Right, all Futures do the same - trying to execute another Future on the same fixed thread pool.

Copy link
Member

Choose a reason for hiding this comment

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

I am worry whether the similar deadlock may occur in other places due to

  • larger parallelism than the fixed thread pool
  • nested parallelism like this

I also realized there is another parmap implementation uses thread pool. Can we use another implemetation?

Copy link
Member Author

Choose a reason for hiding this comment

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

Can we use another implemetation?

This is what @zsxwing proposed. Please, look at my comment #22233 (comment)

Copy link
Member

Choose a reason for hiding this comment

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

Got it. sorry for my overlooking.

Are other places safe where parallelism would not reach the fixed thread pool size?

} else {
logWarning(
s"expected partition column ${partitionNames.head}, but got ${ps(0)}, ignoring it")
Expand All @@ -682,7 +685,7 @@ case class AlterTableRecoverPartitionsCommand(
Seq.empty
}
}
val result = if (partitionNames.length > 1 &&
val result = if (listFilesInParallel && partitionNames.length > 1 &&
statuses.length > threshold || partitionNames.length > 2) {
parmap(statuses)(handleStatus _)
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,23 +52,24 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo
protected override def generateTable(
catalog: SessionCatalog,
name: TableIdentifier,
isDataSource: Boolean = true): CatalogTable = {
isDataSource: Boolean = true,
partitionCols: Seq[String] = Seq("a", "b")): CatalogTable = {
Copy link
Member

Choose a reason for hiding this comment

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

A question about the changes in this file. Are they related to the work of this PR?

Copy link
Member Author

Choose a reason for hiding this comment

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

@gatorsmile Yes, the changes are related to an existing test which was modified to reproduce the issue. In particular, this line is related to support of any number of partition columns.

val storage =
CatalogStorageFormat.empty.copy(locationUri = Some(catalog.defaultTablePath(name)))
val metadata = new MetadataBuilder()
.putString("key", "value")
.build()
val schema = new StructType()
.add("col1", "int", nullable = true, metadata = metadata)
.add("col2", "string")
CatalogTable(
identifier = name,
tableType = CatalogTableType.EXTERNAL,
storage = storage,
schema = new StructType()
.add("col1", "int", nullable = true, metadata = metadata)
.add("col2", "string")
.add("a", "int")
.add("b", "int"),
schema = schema.copy(
fields = schema.fields ++ partitionCols.map(StructField(_, IntegerType))),
provider = Some("parquet"),
partitionColumnNames = Seq("a", "b"),
partitionColumnNames = partitionCols,
createTime = 0L,
createVersion = org.apache.spark.SPARK_VERSION,
tracksPartitionsInCatalog = true)
Expand Down Expand Up @@ -176,7 +177,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
protected def generateTable(
catalog: SessionCatalog,
name: TableIdentifier,
isDataSource: Boolean = true): CatalogTable
isDataSource: Boolean = true,
partitionCols: Seq[String] = Seq("a", "b")): CatalogTable

private val escapedIdentifier = "`(.+)`".r

Expand Down Expand Up @@ -228,8 +230,10 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
private def createTable(
catalog: SessionCatalog,
name: TableIdentifier,
isDataSource: Boolean = true): Unit = {
catalog.createTable(generateTable(catalog, name, isDataSource), ignoreIfExists = false)
isDataSource: Boolean = true,
partitionCols: Seq[String] = Seq("a", "b")): Unit = {
catalog.createTable(
generateTable(catalog, name, isDataSource, partitionCols), ignoreIfExists = false)
}

private def createTablePartition(
Expand Down Expand Up @@ -1131,7 +1135,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
}

test("alter table: recover partition (parallel)") {
withSQLConf("spark.rdd.parallelListingThreshold" -> "1") {
withSQLConf("spark.rdd.parallelListingThreshold" -> "0") {
Copy link
Member

Choose a reason for hiding this comment

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

@MaxGekk, out of curiosity, why does this have to be 0?

Copy link
Member Author

@MaxGekk MaxGekk Aug 25, 2018

Choose a reason for hiding this comment

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

On the recursive calls this condition

val result = if (partitionNames.length > 1 &&
statuses.length > threshold || partitionNames.length > 2) {
is false because statuses.length is 1 and threshold is 1. So, it leads to sequential listening of files. I just enforce parallel scanning even for 1 file/folder.

testRecoverPartitions()
}
}
Expand All @@ -1144,23 +1148,32 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
}

val tableIdent = TableIdentifier("tab1")
createTable(catalog, tableIdent)
val part1 = Map("a" -> "1", "b" -> "5")
createTable(catalog, tableIdent, partitionCols = Seq("a", "b", "c"))
val part1 = Map("a" -> "1", "b" -> "5", "c" -> "19")
createTablePartition(catalog, part1, tableIdent)
assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1))

val part2 = Map("a" -> "2", "b" -> "6")
val part2 = Map("a" -> "2", "b" -> "6", "c" -> "31")
val root = new Path(catalog.getTableMetadata(tableIdent).location)
val fs = root.getFileSystem(spark.sessionState.newHadoopConf())
// valid
fs.mkdirs(new Path(new Path(root, "a=1"), "b=5"))
fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "a.csv")) // file
fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "_SUCCESS")) // file
fs.mkdirs(new Path(new Path(root, "A=2"), "B=6"))
fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "b.csv")) // file
fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "c.csv")) // file
fs.createNewFile(new Path(new Path(root, "A=2/B=6"), ".hiddenFile")) // file
fs.mkdirs(new Path(new Path(root, "A=2/B=6"), "_temporary"))
fs.mkdirs(new Path(new Path(new Path(root, "a=1"), "b=5"), "c=19"))
fs.createNewFile(new Path(new Path(root, "a=1/b=5/c=19"), "a.csv")) // file
fs.createNewFile(new Path(new Path(root, "a=1/b=5/c=19"), "_SUCCESS")) // file

fs.mkdirs(new Path(new Path(new Path(root, "A=2"), "B=6"), "C=31"))
fs.createNewFile(new Path(new Path(root, "A=2/B=6/C=31"), "b.csv")) // file
fs.createNewFile(new Path(new Path(root, "A=2/B=6/C=31"), "c.csv")) // file
fs.createNewFile(new Path(new Path(root, "A=2/B=6/C=31"), ".hiddenFile")) // file
fs.mkdirs(new Path(new Path(root, "A=2/B=6/C=31"), "_temporary"))

val parts = (10 to 100).map { a =>
val part = Map("a" -> a.toString, "b" -> "5", "c" -> "42")
fs.mkdirs(new Path(new Path(new Path(root, s"a=$a"), "b=5"), "c=42"))
fs.createNewFile(new Path(new Path(root, s"a=$a/b=5/c=42"), "a.csv")) // file
createTablePartition(catalog, part, tableIdent)
part
}

// invalid
fs.mkdirs(new Path(new Path(root, "a"), "b")) // bad name
Expand All @@ -1174,7 +1187,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
try {
sql("ALTER TABLE tab1 RECOVER PARTITIONS")
assert(catalog.listPartitions(tableIdent).map(_.spec).toSet ==
Set(part1, part2))
Set(part1, part2) ++ parts)
if (!isUsingHiveMetastore) {
assert(catalog.getPartition(tableIdent, part1).parameters("numFiles") == "1")
assert(catalog.getPartition(tableIdent, part2).parameters("numFiles") == "2")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,8 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA
protected override def generateTable(
catalog: SessionCatalog,
name: TableIdentifier,
isDataSource: Boolean): CatalogTable = {
isDataSource: Boolean,
partitionCols: Seq[String] = Seq("a", "b")): CatalogTable = {
Copy link
Member

Choose a reason for hiding this comment

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

The interface of this function looks strange. The original one is also hacky. We should refine them later.

Copy link
Member

Choose a reason for hiding this comment

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

Yeah, please don't overwrite a method with a default parameter. It's very easy to use different default values then the value to pick up will depend on the type you are using...

val storage =
if (isDataSource) {
val serde = HiveSerDe.sourceToSerDe("parquet")
Expand All @@ -84,17 +85,17 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA
val metadata = new MetadataBuilder()
.putString("key", "value")
.build()
val schema = new StructType()
.add("col1", "int", nullable = true, metadata = metadata)
.add("col2", "string")
CatalogTable(
identifier = name,
tableType = CatalogTableType.EXTERNAL,
storage = storage,
schema = new StructType()
.add("col1", "int", nullable = true, metadata = metadata)
.add("col2", "string")
.add("a", "int")
.add("b", "int"),
schema = schema.copy(
fields = schema.fields ++ partitionCols.map(StructField(_, IntegerType))),
provider = if (isDataSource) Some("parquet") else Some("hive"),
partitionColumnNames = Seq("a", "b"),
partitionColumnNames = partitionCols,
createTime = 0L,
createVersion = org.apache.spark.SPARK_VERSION,
tracksPartitionsInCatalog = true)
Expand Down