[GitHub] [spark] Ngone51 commented on a change in pull request #25612: [SPARK-3137][Core]Replace the global TorrentBroadcast lock with fine grained KeyLock

2019-08-30 Thread GitBox
Ngone51 commented on a change in pull request #25612: [SPARK-3137][Core]Replace 
the global TorrentBroadcast lock with fine grained KeyLock
URL: https://github.com/apache/spark/pull/25612#discussion_r319711054
 
 

 ##
 File path: core/src/main/scala/org/apache/spark/util/KeyLock.scala
 ##
 @@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.util.concurrent.ConcurrentHashMap
+
+/**
+ * A special locking mechanism to provide locking with a given key. By 
providing the same key
+ * (identity is tested using the `equals` method), we ensure there is only one 
`func` running at
+ * the same time.
+ *
+ * @tparam K the type of key to identify a lock. This type must implement 
`equals` and `hashCode`
+ *   correctly as it will be the key type of an internal Map.
+ */
+private[spark] class KeyLock[K] {
+
+  private val lockMap = new ConcurrentHashMap[K, AnyRef]()
+
+  private def acquireLock(key: K): Unit = {
+while (true) {
+  val lock = lockMap.putIfAbsent(key, new Object)
+  if (lock == null) return
+  lock.synchronized {
+while (lockMap.get(key) eq lock) {
 
 Review comment:
   Oh, I see.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] Ngone51 commented on a change in pull request #25612: [SPARK-3137][Core]Replace the global TorrentBroadcast lock with fine grained KeyLock

2019-08-30 Thread GitBox
Ngone51 commented on a change in pull request #25612: [SPARK-3137][Core]Replace 
the global TorrentBroadcast lock with fine grained KeyLock
URL: https://github.com/apache/spark/pull/25612#discussion_r319525047
 
 

 ##
 File path: core/src/main/scala/org/apache/spark/util/KeyLock.scala
 ##
 @@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.util.concurrent.ConcurrentHashMap
+
+/**
+ * A special locking mechanism to provide locking with a given key. By 
providing the same key
+ * (identity is tested using the `equals` method), we ensure there is only one 
`func` running at
+ * the same time.
+ *
+ * @tparam K the type of key to identify a lock. This type must implement 
`equals` and `hashCode`
+ *   correctly as it will be the key type of an internal Map.
+ */
+private[spark] class KeyLock[K] {
+
+  private val lockMap = new ConcurrentHashMap[K, AnyRef]()
+
+  private def acquireLock(key: K): Unit = {
+while (true) {
+  val lock = lockMap.putIfAbsent(key, new Object)
+  if (lock == null) return
+  lock.synchronized {
+while (lockMap.get(key) eq lock) {
 
 Review comment:
   After releasing keylock, if a new thread for the same broadcastId enters to 
put a new object before another queueing thread to check `lockMap.get(key) eq 
lock`,  both threads could get the keylock finally ?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org