|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.kafka010 |
| 19 | + |
| 20 | +import java.{util => ju} |
| 21 | +import java.util.concurrent.ConcurrentHashMap |
| 22 | + |
| 23 | +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener} |
| 24 | +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig} |
| 25 | + |
| 26 | +import org.apache.spark.internal.Logging |
| 27 | + |
| 28 | +/** |
| 29 | + * Provides object pool for objects which is grouped by a key. |
| 30 | + * |
| 31 | + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on |
| 32 | + * the class, and same contract applies: after using the borrowed object, you must either call |
| 33 | + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object |
| 34 | + * should be destroyed. |
| 35 | + * |
| 36 | + * The soft capacity of pool is determined by "poolConfig.capacity" config value, |
| 37 | + * and the pool will have reasonable default value if the value is not provided. |
| 38 | + * (The instance will do its best effort to respect soft capacity but it can exceed when there's |
| 39 | + * a borrowing request and there's neither free space nor idle object to clear.) |
| 40 | + * |
| 41 | + * This class guarantees that no caller will get pooled object once the object is borrowed and |
| 42 | + * not yet returned, hence provide thread-safety usage of non-thread-safe objects unless caller |
| 43 | + * shares the object to multiple threads. |
| 44 | + */ |
| 45 | +private[kafka010] abstract class InternalKafkaConnectorPool[K, V]( |
| 46 | + objectFactory: ObjectFactory[K, V], |
| 47 | + poolConfig: PoolConfig[V], |
| 48 | + swallowedExceptionListener: SwallowedExceptionListener) extends Logging { |
| 49 | + |
| 50 | + // the class is intended to have only soft capacity |
| 51 | + assert(poolConfig.getMaxTotal < 0) |
| 52 | + |
| 53 | + private val pool = { |
| 54 | + val internalPool = new GenericKeyedObjectPool[K, V](objectFactory, poolConfig) |
| 55 | + internalPool.setSwallowedExceptionListener(swallowedExceptionListener) |
| 56 | + internalPool |
| 57 | + } |
| 58 | + |
| 59 | + /** |
| 60 | + * Borrows object from the pool. If there's no idle object for the key, |
| 61 | + * the pool will create the object. |
| 62 | + * |
| 63 | + * If the pool doesn't have idle object for the key and also exceeds the soft capacity, |
| 64 | + * pool will try to clear some of idle objects. |
| 65 | + * |
| 66 | + * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise |
| 67 | + * the object will be kept in pool as active object. |
| 68 | + */ |
| 69 | + def borrowObject(key: K, kafkaParams: ju.Map[String, Object]): V = { |
| 70 | + updateKafkaParamForKey(key, kafkaParams) |
| 71 | + |
| 72 | + if (size >= poolConfig.softMaxSize) { |
| 73 | + logWarning("Pool exceeds its soft max size, cleaning up idle objects...") |
| 74 | + pool.clearOldest() |
| 75 | + } |
| 76 | + |
| 77 | + pool.borrowObject(key) |
| 78 | + } |
| 79 | + |
| 80 | + /** Returns borrowed object to the pool. */ |
| 81 | + def returnObject(connector: V): Unit = { |
| 82 | + pool.returnObject(createKey(connector), connector) |
| 83 | + } |
| 84 | + |
| 85 | + /** Invalidates (destroy) borrowed object to the pool. */ |
| 86 | + def invalidateObject(connector: V): Unit = { |
| 87 | + pool.invalidateObject(createKey(connector), connector) |
| 88 | + } |
| 89 | + |
| 90 | + /** Invalidates all idle values for the key */ |
| 91 | + def invalidateKey(key: K): Unit = { |
| 92 | + pool.clear(key) |
| 93 | + } |
| 94 | + |
| 95 | + /** |
| 96 | + * Closes the keyed object pool. Once the pool is closed, |
| 97 | + * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject |
| 98 | + * will continue to work, with returned objects destroyed on return. |
| 99 | + * |
| 100 | + * Also destroys idle instances in the pool. |
| 101 | + */ |
| 102 | + def close(): Unit = { |
| 103 | + pool.close() |
| 104 | + } |
| 105 | + |
| 106 | + def reset(): Unit = { |
| 107 | + // this is the best-effort of clearing up. otherwise we should close the pool and create again |
| 108 | + // but we don't want to make it "var" only because of tests. |
| 109 | + pool.clear() |
| 110 | + } |
| 111 | + |
| 112 | + def numIdle: Int = pool.getNumIdle |
| 113 | + |
| 114 | + def numIdle(key: K): Int = pool.getNumIdle(key) |
| 115 | + |
| 116 | + def numActive: Int = pool.getNumActive |
| 117 | + |
| 118 | + def numActive(key: K): Int = pool.getNumActive(key) |
| 119 | + |
| 120 | + def size: Int = numIdle + numActive |
| 121 | + |
| 122 | + def size(key: K): Int = numIdle(key) + numActive(key) |
| 123 | + |
| 124 | + private def updateKafkaParamForKey(key: K, kafkaParams: ju.Map[String, Object]): Unit = { |
| 125 | + // We can assume that kafkaParam should not be different for same cache key, |
| 126 | + // otherwise we can't reuse the cached object and cache key should contain kafkaParam. |
| 127 | + // So it should be safe to put the key/value pair only when the key doesn't exist. |
| 128 | + val oldKafkaParams = objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams) |
| 129 | + require(oldKafkaParams == null || kafkaParams == oldKafkaParams, "Kafka parameters for same " + |
| 130 | + s"cache key should be equal. old parameters: $oldKafkaParams new parameters: $kafkaParams") |
| 131 | + } |
| 132 | + |
| 133 | + protected def createKey(connector: V): K |
| 134 | +} |
| 135 | + |
| 136 | +private[kafka010] abstract class PoolConfig[V] extends GenericKeyedObjectPoolConfig[V] { |
| 137 | + |
| 138 | + init() |
| 139 | + |
| 140 | + def softMaxSize: Int |
| 141 | + |
| 142 | + def jmxEnabled: Boolean |
| 143 | + |
| 144 | + def minEvictableIdleTimeMillis: Long |
| 145 | + |
| 146 | + def evictorThreadRunIntervalMillis: Long |
| 147 | + |
| 148 | + def jmxNamePrefix: String |
| 149 | + |
| 150 | + def init(): Unit = { |
| 151 | + // NOTE: Below lines define the behavior, so do not modify unless you know what you are |
| 152 | + // doing, and update the class doc accordingly if necessary when you modify. |
| 153 | + |
| 154 | + // 1. Set min idle objects per key to 0 to avoid creating unnecessary object. |
| 155 | + // 2. Set max idle objects per key to 3 but set total objects per key to infinite |
| 156 | + // which ensures borrowing per key is not restricted. |
| 157 | + // 3. Set max total objects to infinite which ensures all objects are managed in this pool. |
| 158 | + setMinIdlePerKey(0) |
| 159 | + setMaxIdlePerKey(3) |
| 160 | + setMaxTotalPerKey(-1) |
| 161 | + setMaxTotal(-1) |
| 162 | + |
| 163 | + // Set minimum evictable idle time which will be referred from evictor thread |
| 164 | + setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis) |
| 165 | + setSoftMinEvictableIdleTimeMillis(-1) |
| 166 | + |
| 167 | + // evictor thread will run test with ten idle objects |
| 168 | + setTimeBetweenEvictionRunsMillis(evictorThreadRunIntervalMillis) |
| 169 | + setNumTestsPerEvictionRun(10) |
| 170 | + setEvictionPolicy(new DefaultEvictionPolicy[V]()) |
| 171 | + |
| 172 | + // Immediately fail on exhausted pool while borrowing |
| 173 | + setBlockWhenExhausted(false) |
| 174 | + |
| 175 | + setJmxEnabled(jmxEnabled) |
| 176 | + setJmxNamePrefix(jmxNamePrefix) |
| 177 | + } |
| 178 | +} |
| 179 | + |
| 180 | +private[kafka010] abstract class ObjectFactory[K, V] extends BaseKeyedPooledObjectFactory[K, V] { |
| 181 | + val keyToKafkaParams = new ConcurrentHashMap[K, ju.Map[String, Object]]() |
| 182 | + |
| 183 | + override def create(key: K): V = { |
| 184 | + Option(keyToKafkaParams.get(key)) match { |
| 185 | + case Some(kafkaParams) => createValue(key, kafkaParams) |
| 186 | + case None => throw new IllegalStateException("Kafka params should be set before " + |
| 187 | + "borrowing object.") |
| 188 | + } |
| 189 | + } |
| 190 | + |
| 191 | + override def wrap(value: V): PooledObject[V] = { |
| 192 | + new DefaultPooledObject[V](value) |
| 193 | + } |
| 194 | + |
| 195 | + protected def createValue(key: K, kafkaParams: ju.Map[String, Object]): V |
| 196 | +} |
| 197 | + |
| 198 | +private[kafka010] class CustomSwallowedExceptionListener(connectorType: String) |
| 199 | + extends SwallowedExceptionListener with Logging { |
| 200 | + override def onSwallowException(e: Exception): Unit = { |
| 201 | + logError(s"Error closing Kafka $connectorType", e) |
| 202 | + } |
| 203 | +} |
0 commit comments