Skip to content

Conversation

@zuotingbing
Copy link

@zuotingbing zuotingbing commented Dec 20, 2017

What changes were proposed in this pull request?

Currently we do not call the super.init(hiveConf) in SparkSQLSessionManager.init. So we do not load the config HIVE_SERVER2_SESSION_CHECK_INTERVAL HIVE_SERVER2_IDLE_SESSION_TIMEOUT HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION , which cause the session timeout checker does not work.

How was this patch tested?

manual tests

Copy link
Member

@felixcheung felixcheung left a comment

Choose a reason for hiding this comment

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

}
}

private void initSessionTimeoutCheckerConfig() {
Copy link
Member

Choose a reason for hiding this comment

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

should we refactor createBackgroundOperationPool to call this?

Copy link
Author

Choose a reason for hiding this comment

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

Thanks @felixcheung . In my opinion you can create a new or follow-up PR if it is necessary. This PR is to fix the bug about the Session Timeout Checker does not work currently.

Copy link
Member

Choose a reason for hiding this comment

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

@felixcheung
Copy link
Member

Jenkins, ok to test

@SparkQA
Copy link

SparkQA commented Dec 22, 2017

Test build #85299 has finished for PR 20025 at commit 34b7680.

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

@zuotingbing
Copy link
Author

zuotingbing commented Dec 27, 2017

Could you please merge this to Master? Thanks @felixcheung @srowen

Copy link
Member

@felixcheung felixcheung left a comment

Choose a reason for hiding this comment

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

@cloud-fan @gatorsmile would you like a look please?

}
}

private void initSessionTimeoutCheckerConfig() {
Copy link
Member

Choose a reason for hiding this comment

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

@zuotingbing
Copy link
Author

zuotingbing commented Dec 28, 2017

Currently we do not call the super.init(hiveConf) in SparkSQLSessionManager.init, it means SessionManager.init which includes createBackgroundOperationPool() will not be called.

i fix this bug refer to if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { invoke(classOf[SessionManager], this, "initOperationLogRootDir") } or setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) which also include duplicate lines in SessionManager.java

@liufengdb
Copy link

liufengdb commented Dec 29, 2017

@zuotingbing I think all the code in SparkSQLSessionManager.scala should be gone because they are just some reflection hacks. It is possible to call super.init(hiveConf) instead to get the session timeout checker and all other things start, isn't?

@zuotingbing
Copy link
Author

@liufengdb I think the class SessionManager.java is merged from Hive originally, and in Spark we redesigned it by adding SparkSQLSessionManager.scala with no affect to SessionManager.java :
val sparkSqlSessionManager = new SparkSQLSessionManager(hiveServer, sqlContext) setSuperField(this, "sessionManager", sparkSqlSessionManager)

@zuotingbing
Copy link
Author

@rxin Could you please to review this? Thanks.

In my opinion we can create a new or follow-up PR if refactor is necessary. This PR is to fix the bug about the Session Timeout Checker does not work currently.

@liufengdb
Copy link

liufengdb commented Dec 29, 2017

My understanding is that the reflection was used originally because we don't control what is done inside the init method of the super class, given that different hive versions can be used. However, after we inlined the hive code, it is safe to call the init method directly now. This is a cleaner way to fix the referred and other potential bugs, IMO.

@zuotingbing
Copy link
Author

@gatorsmile What is your opinion about this? Thanks!

@zuotingbing
Copy link
Author

zuotingbing commented Jan 16, 2018

ping @gatorsmile

@zuotingbing
Copy link
Author

ping @liufengdb

@SparkQA
Copy link

SparkQA commented Jan 16, 2018

Test build #86167 has finished for PR 20025 at commit 162a93f.

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

@SparkQA
Copy link

SparkQA commented Jan 17, 2018

Test build #86231 has finished for PR 20025 at commit 79fc11a.

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

@zuotingbing
Copy link
Author

@cloud-fan @gatorsmile @liufengdb @felixcheung @srowen @vanzin Is anybody could make further contact and discuss this PR? Thanks!

keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(poolQueueSize),
new ThreadFactoryWithGarbageCleanup(threadPoolName));
backgroundOperationPool.allowCoreThreadTimeOut(true);

Choose a reason for hiding this comment

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

I think we can keep this file unchanged. Looks like the hive thread pool is more flexible than the spark one.

Copy link
Author

Choose a reason for hiding this comment

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

yes, i will update it . Thanks!

@liufengdb
Copy link

@gatorsmile @felixcheung I left one comment, otherwise lgtm.

@gatorsmile
Copy link
Member

ok to test

@SparkQA
Copy link

SparkQA commented Jan 18, 2018

Test build #86349 has finished for PR 20025 at commit 79fc11a.

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

…ead pool is more flexible than the spark one.
@SparkQA
Copy link

SparkQA commented Jan 19, 2018

Test build #86370 has finished for PR 20025 at commit 5b3c06f.

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

import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
Copy link
Member

Choose a reason for hiding this comment

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

revert this back.

Copy link
Author

Choose a reason for hiding this comment

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

yes, thanks

}
createBackgroundOperationPool();
addService(operationManager);
super.init(hiveConf);

Choose a reason for hiding this comment

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

hmm, I think we revert keep this line too.

Copy link
Author

Choose a reason for hiding this comment

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

yes, we could keep super.init() here and remove initCompositeService in SparkSQLSessionManager.init(). thanks

@SparkQA
Copy link

SparkQA commented Jan 23, 2018

Test build #86508 has finished for PR 20025 at commit 883deb2.

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

@zuotingbing
Copy link
Author

@gatorsmile @liufengdb Anything else? Thanks!

@liufengdb
Copy link

lgtm

Copy link
Member

@felixcheung felixcheung left a comment

Choose a reason for hiding this comment

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

LGTM so long as @gatorsmile approves.

@gatorsmile
Copy link
Member

Thanks! Merged to master/2.3

asfgit pushed a commit that referenced this pull request Jan 24, 2018
…nager.

## What changes were proposed in this pull request?

Currently we do not call the `super.init(hiveConf)` in `SparkSQLSessionManager.init`. So we do not load the config `HIVE_SERVER2_SESSION_CHECK_INTERVAL HIVE_SERVER2_IDLE_SESSION_TIMEOUT HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION` , which cause the session timeout checker does not work.

## How was this patch tested?

manual tests

Author: zuotingbing <[email protected]>

Closes #20025 from zuotingbing/SPARK-22837.

(cherry picked from commit bbb87b3)
Signed-off-by: gatorsmile <[email protected]>
@asfgit asfgit closed this in bbb87b3 Jan 24, 2018
@zuotingbing zuotingbing deleted the SPARK-22837 branch January 25, 2018 02:33
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.

5 participants