-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-34573][SQL] Avoid global locking in SQLConf object for sqlConfEntries map #31689
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
Conversation
… lock
Note that merge function is only called if old value is not null, so it will always fail the require check (as intended):
V newValue = (oldValue == null) ? value :
remappingFunction.apply(oldValue, value);
|
ok to test |
|
This change could get away the lock wait you mentioned in the jira? If so, could you describe more in the PR description? |
Added more descriptive PR text. |
|
Kubernetes integration test starting |
|
Kubernetes integration test status failure |
| private[sql] val sqlConfEntries = java.util.Collections.synchronizedMap( | ||
| new java.util.HashMap[String, ConfigEntry[_]]()) | ||
| private[sql] val sqlConfEntries = | ||
| new ConcurrentHashMap[String, ConfigEntry[_]]() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Seems a reasonable fix cc: @srowen @cloud-fan @viirya
srowen
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK pending tests
viirya
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
looks reasonable. pending test.
@srowen what does this mean? in jenkins the failure is: |
|
Test build #135617 has finished for PR 31689 at commit
|
The failure is not related to this PR, so it's okay just to ignore it. |
|
Thanks! Merged to master. |
|
Yeah seems good |
|
late LGTM |
What changes were proposed in this pull request?
In the
SQLConfobject, thesqlConfEntriesmap is globally synchronized (it is a JavaCollections.synchronizedMap): any operation, including a get, will need to acquire the lock.An example of this is calling the
DatatType.sameTypemethod. This will trigger a check onSQLConf.get.caseSensitiveAnalysis. So every time we compare two datatypes with sameType, we hit a lock.To avoid having multiple tasks locking on this, a better approach would be to use a map that does not lock on read (like a
ConcurrentHashMap). This map implementation does not lock on read, and on write it only locks the map partially. The only lock that happens is on write on the same map key.Why are the changes needed?
Multiple tasks performing any operation that directly or indirectly trigger a query to the
SQLConf.sqlConfEntriesmap, will require acquiring a global lock on that map. Something as easy as callingDataType.sameType(...)would be locking on the globalsqlConfEntrieslock of theCollections.synchronizedMap.Does this PR introduce any user-facing change?
No
How was this patch tested?
No functionality change. Existing unit tests run normally.