Skip to content

Conversation

@ericl
Copy link
Contributor

@ericl ericl commented Oct 31, 2016

What changes were proposed in this pull request?

There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive.

(1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition.
(2) INSERT|OVERWRITE does not work with partitions that have custom locations.

This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when manageFilesourcePartitions = false is unchanged.

There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release.

How was this patch tested?

Unit tests.

@ericl
Copy link
Contributor Author

ericl commented Oct 31, 2016

cc @cloud-fan @yhuai

@SparkQA
Copy link

SparkQA commented Oct 31, 2016

Test build #67846 has finished for PR 15705 at commit 669e6cc.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 1, 2016

Test build #67848 has finished for PR 15705 at commit fec7c9e.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 1, 2016

Test build #67849 has finished for PR 15705 at commit 07c6787.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

val path = new Path(p.storage.locationUri.get)
val fs = path.getFileSystem(hadoopConf)
PartitionPath(
p.toRow(partitionSchema), path.makeQualified(fs.getUri, fs.getWorkingDirectory))
Copy link
Contributor

Choose a reason for hiding this comment

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

why this change? Doesn't new Path qualify the path string?

Copy link
Contributor Author

@ericl ericl Nov 1, 2016

Choose a reason for hiding this comment

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

Apparently not. The unit test actually fails if you do that, since the path seems to be missing the file: prefix and we fail to find the files in the partition.

Copy link
Contributor

Choose a reason for hiding this comment

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

But we will qualify it before writing to it at here, doesn't it work?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The issue is that the user can store arbitrary string paths with ALTER TABLE PARTITIONS SET LOCATION. Therefore, we must manually qualify the locations that come from the catalog or else they might not necessarily match up with the paths read from the filesystem.

"Cannot overwrite a path that is also being read from.")
}

val overwritePartitionPath = if (overwrite.specificPartition.isDefined &&
Copy link
Contributor

@cloud-fan cloud-fan Nov 1, 2016

Choose a reason for hiding this comment

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

can we just pass the partition path as outputPath to InsertIntoHadoopFsRelationCommand and set partition columns to Nil? then we don't need to add an extra parameter to InsertIntoHadoopFsRelationCommand

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, that seems a little cleaner.

@SparkQA
Copy link

SparkQA commented Nov 1, 2016

Test build #67864 has finished for PR 15705 at commit 0daff74.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 1, 2016

Test build #67911 has finished for PR 15705 at commit ca31f35.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class TaskCommitMessage(obj: Any) extends Serializable
    • abstract class FileCommitProtocol
    • class HadoopCommitProtocolWrapper(path: String, isAppend: Boolean)
    • logInfo(s\"Using user defined output committer class $
    • logInfo(s\"Using output committer class $

OverwriteOptions(
overwrite,
if (overwrite && partition.nonEmpty) {
Some(partition.map(kv => (kv._1, kv._2.get)))
Copy link
Contributor

Choose a reason for hiding this comment

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

do we need to consider dynamic partition here?

@ericl
Copy link
Contributor Author

ericl commented Nov 2, 2016

I think you don't have to since this is just the test suite.

On Tue, Nov 1, 2016, 8:49 PM Wenchen Fan [email protected] wrote:

@cloud-fan commented on this pull request.

In
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
#15705 (review):

@@ -180,7 +180,16 @@ class PlanParserSuite extends PlanTest {
partition: Map[String, Option[String]],
overwrite: Boolean = false,
ifNotExists: Boolean = false): LogicalPlan =

  •  InsertIntoTable(table("s"), partition, plan, overwrite, ifNotExists)
    
  •  InsertIntoTable(
    
  •    table("s"), partition, plan,
    
  •    OverwriteOptions(
    
  •      overwrite,
    
  •      if (overwrite && partition.nonEmpty) {
    
  •        Some(partition.map(kv => (kv._1, kv._2.get)))
    

do we need to consider dynamic partition here?


You are receiving this because you authored the thread.
Reply to this email directly, view it on GitHub
#15705 (review),
or mute the thread
https://github.com/notifications/unsubscribe-auth/AAA6SpEB-R5k4u55HP0Wt6z9i1OihpNSks5q6AhJgaJpZM4KlpEq
.

@cloud-fan
Copy link
Contributor

LGTM, merging to master!

@asfgit asfgit closed this in abefe2e Nov 2, 2016
asfgit pushed a commit that referenced this pull request Nov 2, 2016
…ITION for Datasource tables

There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive.

(1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition.
(2) INSERT|OVERWRITE does not work with partitions that have custom locations.

This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when `manageFilesourcePartitions = false` is unchanged.

There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release.

Unit tests.

Author: Eric Liang <[email protected]>

Closes #15705 from ericl/sc-4942.

(cherry picked from commit abefe2e)
Signed-off-by: Reynold Xin <[email protected]>
asfgit pushed a commit that referenced this pull request Dec 14, 2016
## What changes were proposed in this pull request?

`OverwriteOptions` was introduced in #15705, to carry the information of static partitions. However, after further refactor, this information becomes duplicated and we can remove `OverwriteOptions`.

## How was this patch tested?

N/A

Author: Wenchen Fan <[email protected]>

Closes #15995 from cloud-fan/overwrite.
robert3005 pushed a commit to palantir/spark that referenced this pull request Dec 15, 2016
## What changes were proposed in this pull request?

`OverwriteOptions` was introduced in apache#15705, to carry the information of static partitions. However, after further refactor, this information becomes duplicated and we can remove `OverwriteOptions`.

## How was this patch tested?

N/A

Author: Wenchen Fan <[email protected]>

Closes apache#15995 from cloud-fan/overwrite.
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…ITION for Datasource tables

## What changes were proposed in this pull request?

There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive.

(1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition.
(2) INSERT|OVERWRITE does not work with partitions that have custom locations.

This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when `manageFilesourcePartitions = false` is unchanged.

There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release.

## How was this patch tested?

Unit tests.

Author: Eric Liang <[email protected]>

Closes apache#15705 from ericl/sc-4942.
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
## What changes were proposed in this pull request?

`OverwriteOptions` was introduced in apache#15705, to carry the information of static partitions. However, after further refactor, this information becomes duplicated and we can remove `OverwriteOptions`.

## How was this patch tested?

N/A

Author: Wenchen Fan <[email protected]>

Closes apache#15995 from cloud-fan/overwrite.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants