Skip to content

Conversation

@sujith71955
Copy link
Contributor

@sujith71955 sujith71955 commented Sep 27, 2018

What changes were proposed in this pull request?

As part of insert command in FileFormatWriter, a job context is created for handling the write operation , While initializing the job context using setupJob() API in HadoopMapReduceCommitProtocol , we set the jobid in the Jobcontext configuration.In FileFormatWriter since we are directly getting the jobId from the map reduce JobContext the job id will come as null while adding the log. As a solution we shall get the jobID from the configuration of the map reduce Jobcontext.

How was this patch tested?

Manually, verified the logs after the changes.

spark-25521 1

@sujith71955
Copy link
Contributor Author

sujith71955 commented Sep 27, 2018

cc @cloud-fan @srowen
Please review and let me know for any suggestions. Thanks

@srowen
Copy link
Member

srowen commented Sep 27, 2018

Is the value logged here always null?
I am not sure if it's meaningful to log mapreduce.job.id, especially given its name. If there's no meaningful job ID here do we are about it at all? how about deleting the log?
SparkHadoopWriter does something similar.

@sujith71955
Copy link
Contributor Author

sujith71955 commented Sep 27, 2018

Is the value logged here always null? - Yes its always printing null.
I am not sure if it's meaningful to log mapreduce.job.id, especially given its name. If there's no meaningful job ID here do we are about it at all? how about deleting the log?
SparkHadoopWriter does something similar.

even initially i thought the same, not sure whether mapreduce.job.id makes sense here, but i think we shall not display null . Deleting the log will be the easiest option but just curious to know why the author is trying to log a map reduce job id .

@sujith71955
Copy link
Contributor Author

cc @gatorsmile

Copy link
Contributor

Choose a reason for hiding this comment

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

we should log something here, but mapreduce.job.id is not useful, how about description.uuid?

Copy link
Member

Choose a reason for hiding this comment

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

SparkHadoopWriter needs a similar change, then, BTW

Copy link
Contributor Author

@sujith71955 sujith71955 Sep 28, 2018

Choose a reason for hiding this comment

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

Thanks for the suggestions!! I will update this PR.
@cloud-fan Yes, i think displaying description.uuid makes more sense as the user can get to know about their particular write job status.
I will also update the message with Write Job instead of Job, Hope thats fine.

@sujith71955
Copy link
Contributor Author

@srowen @cloud-fan
I was testing the SparkHadoopWriter flow, with below steps and i could see in the log with job id printed properly, so is it fine to update this flow also with description.uuid ? Attaching the snapshot of logs based SparkHadoopWriter flow
val rdd=spark.sparkContext.newAPIHadoopFile("D:/data/x.csv",classOf[org.apache.hadoop.mapreduce.lib.input.NLineInputFormat],classOf[org.apache.hadoop.io.LongWritable],classOf[org.apache.hadoop.io.Text])

val hconf=spark.sparkContext.hadoopConfiguration

hconf.set("mapreduce.output.fileoutputformat.outputdir","D:/data/test")

scala> rdd.saveAsNewAPIHadoopDataset(hconf)

sparkhadoopwriter

@sujith71955
Copy link
Contributor Author

sujith71955 commented Oct 3, 2018

When i digged the code i could see in SparkHadoopWriter, while creating job context itself job id is been intialized. Let me know for any suggestions.

image

@cloud-fan
Copy link
Contributor

Can we update the PR to use description.uuid first?

…mmand Job is finished.

## What changes were proposed in this pull request?
As part of  insert command  in FileFormatWriter, a job context is created for handling the write operation , While initializing the job context setupJob() API
in HadoopMapReduceCommitProtocol sets the jobid  in the Jobcontext configuration, Since we are directly getting the jobId from the map reduce JobContext the job id will come as
null in the logs. As a solution we shall get the jobID from the configuration of the map reduce Jobcontext

## How was this patch tested?
Manually, verified the logs after the changes.
@sujith71955
Copy link
Contributor Author

Can we update the PR to use description.uuid first?

Updated FileFormatWriter with description.uuid, attaching the verification snapshot .
image

@sujith71955
Copy link
Contributor Author

Can we update the PR to use description.uuid first?

Updated FileFormatWriter with description.uuid, attaching the verification snapshot .
image

Let me know whether we shall update SparkHadoopWriter.scala flow as in this flow currently the jobid is been displayed properly , to display the job description uuid i need to explore as this flow doesnt holds any WriteJobDescription instance.

@cloud-fan
Copy link
Contributor

ok to test

@SparkQA
Copy link

SparkQA commented Oct 4, 2018

Test build #96925 has finished for PR 22572 at commit 56c5ff5.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

retest this please

@SparkQA
Copy link

SparkQA commented Oct 4, 2018

Test build #96938 has finished for PR 22572 at commit 56c5ff5.

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

@asfgit asfgit closed this in 4597007 Oct 5, 2018
@cloud-fan
Copy link
Contributor

thanks, merging to master/2.4!

asfgit pushed a commit that referenced this pull request Oct 5, 2018
…ommand Job is finished.

## What changes were proposed in this pull request?
``As part of  insert command  in FileFormatWriter, a job context is created for handling the write operation , While initializing the job context using setupJob() API
in HadoopMapReduceCommitProtocol , we set the jobid  in the Jobcontext configuration.In FileFormatWriter since we are directly getting the jobId from the map reduce JobContext the job id will come as null  while adding the log. As a solution we shall get the jobID from the configuration of the map reduce Jobcontext.``

## How was this patch tested?
Manually, verified the logs after the changes.

![spark-25521 1](https://user-images.githubusercontent.com/12999161/46164933-e95ab700-c2ac-11e8-88e9-49fa5100b872.PNG)

Closes #22572 from sujith71955/master_log_issue.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 4597007)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
…ommand Job is finished.

## What changes were proposed in this pull request?
``As part of  insert command  in FileFormatWriter, a job context is created for handling the write operation , While initializing the job context using setupJob() API
in HadoopMapReduceCommitProtocol , we set the jobid  in the Jobcontext configuration.In FileFormatWriter since we are directly getting the jobId from the map reduce JobContext the job id will come as null  while adding the log. As a solution we shall get the jobID from the configuration of the map reduce Jobcontext.``

## How was this patch tested?
Manually, verified the logs after the changes.

![spark-25521 1](https://user-images.githubusercontent.com/12999161/46164933-e95ab700-c2ac-11e8-88e9-49fa5100b872.PNG)

Closes apache#22572 from sujith71955/master_log_issue.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
zzcclp added a commit to zzcclp/spark that referenced this pull request Sep 20, 2019
…to command Job is finished. apache#22572

As part of insert command in FileFormatWriter, a job context is created for handling the write operation , While initializing the job context using setupJob() API in HadoopMapReduceCommitProtocol , we set the jobid in the Jobcontext configuration.In FileFormatWriter since we are directly getting the jobId from the map reduce JobContext the job id will come as null while adding the log. As a solution we shall get the jobID from the configuration of the map reduce Jobcontext.
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.

4 participants