Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-32772][SQL] Reduce log messages for spark-sql CLI #29619

Closed
wants to merge 2 commits into from

Conversation

sarutak
Copy link
Member

@sarutak sarutak commented Sep 1, 2020

What changes were proposed in this pull request?

This PR reduces log messages for spark-sql CLI like spark-shell and pyspark CLI.

Why are the changes needed?

When we launch spark-sql CLI, too many log messages are shown and it's sometimes difficult to find the result of query.

spark-sql> SELECT now();
20/09/02 00:11:45 INFO CodeGenerator: Code generated in 10.121625 ms
20/09/02 00:11:45 INFO SparkContext: Starting job: main at NativeMethodAccessorImpl.java:0
20/09/02 00:11:45 INFO DAGScheduler: Got job 0 (main at NativeMethodAccessorImpl.java:0) with 1 output partitions
20/09/02 00:11:45 INFO DAGScheduler: Final stage: ResultStage 0 (main at NativeMethodAccessorImpl.java:0)
20/09/02 00:11:45 INFO DAGScheduler: Parents of final stage: List()
20/09/02 00:11:45 INFO DAGScheduler: Missing parents: List()
20/09/02 00:11:45 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0), which has no missing parents
20/09/02 00:11:45 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 6.3 KiB, free 366.3 MiB)
20/09/02 00:11:45 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 3.2 KiB, free 366.3 MiB)
20/09/02 00:11:45 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 192.168.1.204:42615 (size: 3.2 KiB, free: 366.3 MiB)
20/09/02 00:11:45 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1348
20/09/02 00:11:45 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[2] at main at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0))
20/09/02 00:11:45 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0
20/09/02 00:11:45 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (192.168.1.204, executor driver, partition 0, PROCESS_LOCAL, 7561 bytes) taskResourceAssignments Map()
20/09/02 00:11:45 INFO Executor: Running task 0.0 in stage 0.0 (TID 0)
20/09/02 00:11:45 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 1446 bytes result sent to driver
20/09/02 00:11:45 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 238 ms on 192.168.1.204 (executor driver) (1/1)
20/09/02 00:11:45 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool 
20/09/02 00:11:45 INFO DAGScheduler: ResultStage 0 (main at NativeMethodAccessorImpl.java:0) finished in 0.343 s
20/09/02 00:11:45 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job
20/09/02 00:11:45 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished
20/09/02 00:11:45 INFO DAGScheduler: Job 0 finished: main at NativeMethodAccessorImpl.java:0, took 0.377489 s
2020-09-02 00:11:45.07
Time taken: 0.704 seconds, Fetched 1 row(s)
20/09/02 00:11:45 INFO SparkSQLCLIDriver: Time taken: 0.704 seconds, Fetched 1 row(s)

Does this PR introduce any user-facing change?

Yes. Log messages are reduced for spark-sql CLI like as follows.

20/09/02 00:34:51 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/09/02 00:34:53 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
20/09/02 00:34:53 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
20/09/02 00:34:55 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
20/09/02 00:34:55 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore kou@192.168.1.204
Spark master: local[*], Application Id: local-1598974492822
spark-sql> SELECT now();
2020-09-02 00:35:05.258
Time taken: 2.299 seconds, Fetched 1 row(s)

How was this patch tested?

Launched spark-sql CLI and confirmed that log messages are reduced as I paste above.

@SparkQA
Copy link

SparkQA commented Sep 1, 2020

Test build #128163 has finished for PR 29619 at commit 53c0798.

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

Copy link
Member

@srowen srowen left a comment

Choose a reason for hiding this comment

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

Does this match how spark-shell is configured? seems fine then.

@@ -54,6 +54,8 @@ import org.apache.spark.util.ShutdownHookManager
* has dropped its support.
*/
private[hive] object SparkSQLCLIDriver extends Logging {
initializeLogIfNecessary(true)

Copy link
Member

Choose a reason for hiding this comment

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

Probably best to move this to line 63

@sarutak
Copy link
Member Author

sarutak commented Sep 2, 2020

Does this match how spark-shell is configured? seems fine then.

I think so. spark-shell, pyspark and sparkr CLI seem to be configured with the same manner.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM. Thank you, @sarutak and @srowen .
I also verified locally. It works consistently with the other shells.
Merged to master for Apache Spark 3.1.0 on December 2020.

wangyum added a commit that referenced this pull request Sep 4, 2020
…ark-sql CLI

### What changes were proposed in this pull request?

Remove legacy silent support mode for spark-sql CLI.

### Why are the changes needed?

#29619 add new silent mode. We can remove legacy silent support mode.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Manual test:
```
spark-sql> LM-SHC-16508156:spark yumwang$ bin/spark-sql -S
NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of assembly.
20/09/03 09:06:12 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
20/09/03 09:06:16 WARN HiveConf: HiveConf of name hive.stats.jdbc.timeout does not exist
20/09/03 09:06:16 WARN HiveConf: HiveConf of name hive.stats.retries.wait does not exist
20/09/03 09:06:19 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0
20/09/03 09:06:19 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore yumwang10.226.196.190
spark-sql> select * from test1;
1
spark-sql> select * from test1;
1

```

Closes #29631 from wangyum/SPARK-32772.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
4 participants