[
https://issues.apache.org/jira/browse/MAPREDUCE-7403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17581529#comment-17581529
]
ASF GitHub Bot commented on MAPREDUCE-7403:
-------------------------------------------
attilapiros commented on code in PR #4728:
URL: https://github.com/apache/hadoop/pull/4728#discussion_r949572501
##########
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestManifestCommitProtocol.java:
##########
@@ -1549,6 +1550,21 @@ public void testOutputFormatIntegration() throws
Throwable {
ManifestCommitter committer = (ManifestCommitter)
outputFormat.getOutputCommitter(tContext);
+ // check path capabilities directly
+ Assertions.assertThat(committer.hasCapability(
+ ManifestCommitterConstants.CAPABILITY_DYNAMIC_PARTITIONING))
+ .describedAs("dynamic partitioning capability in committer %s",
+ committer);
+ // and through a binding committer -passthrough is critical
+ // for the spark binding.
+ BindingPathOutputCommitter bindingCommitter =
+ new BindingPathOutputCommitter(outputDir, tContext);
+ Assertions.assertThat(bindingCommitter.hasCapability(
+ ManifestCommitterConstants.CAPABILITY_DYNAMIC_PARTITIONING))
+ .describedAs("dynamic partitioning capability in committer %s",
+ bindingCommitter);
Review Comment:
```suggestion
bindingCommitter).isTrue();
```
##########
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestManifestCommitProtocol.java:
##########
@@ -1549,6 +1550,21 @@ public void testOutputFormatIntegration() throws
Throwable {
ManifestCommitter committer = (ManifestCommitter)
outputFormat.getOutputCommitter(tContext);
+ // check path capabilities directly
+ Assertions.assertThat(committer.hasCapability(
+ ManifestCommitterConstants.CAPABILITY_DYNAMIC_PARTITIONING))
+ .describedAs("dynamic partitioning capability in committer %s",
+ committer);
Review Comment:
```suggestion
committer).isTrue();
```
> Support spark dynamic partitioning in the Manifest Committer
> ------------------------------------------------------------
>
> Key: MAPREDUCE-7403
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-7403
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Components: mrv2
> Affects Versions: 3.3.9
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Priority: Major
> Labels: pull-request-available
>
> Currently the spark integration with PathOutputCommitters rejects attempt to
> instantiate them if dynamic partitioning is enabled. That is because the
> spark partitioning code assumes that
> # file rename works as a fast and safe commit algorithm
> # the working directory is in the same FS as the final directory
> Assumption 1 doesn't hold on s3a, and #2 isn't true for the staging
> committers.
> The new abfs/gcs manifest committer and the target stores do meet both
> requirements. So we no longer need to reject the operation, provided the
> spark side binding-code can can identify when all is good.
> Proposed: add a new hasCapability() probe which, if, a committer implements
> StreamCapabilities can be used to see if the committer will work.
> ManifestCommitter will declare that it holds. As the API has existed since
> 2.10, it will be immediately available.
> spark's PathOutputCommitProtocol to query the committer in setupCommitter,
> and fail if dynamicPartitionOverwrite is requested but not available.
> BindingParquetOutputCommitter to implement and forward
> StreamCapabilities.hasCapability.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]